From e44e1ad0d4cb7b08e4b1de3cf863f060e4d493c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jun 2020 21:57:38 +0300 Subject: [PATCH 0001/1911] Don't ignore duplicate parts written to replicas --- src/Storages/MergeTree/MergeTreeData.cpp | 27 +- src/Storages/MergeTree/MergeTreeData.h | 5 +- .../ReplicatedMergeTreeBlockOutputStream.cpp | 357 ++++++++++-------- .../01319_manual_write_to_replicas.reference | 6 + .../01319_manual_write_to_replicas.sql | 25 ++ 5 files changed, 262 insertions(+), 158 deletions(-) create mode 100644 tests/queries/0_stateless/01319_manual_write_to_replicas.reference create mode 100644 tests/queries/0_stateless/01319_manual_write_to_replicas.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 023e67ec3de..62059507b77 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1754,16 +1754,27 @@ MergeTreeData::DataPartsVector MergeTreeData::getActivePartsToReplace( } -void MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction) +bool MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction) { - auto removed = renameTempPartAndReplace(part, increment, out_transaction); - if (!removed.empty()) - throw Exception("Added part " + part->name + " covers " + toString(removed.size()) - + " existing part(s) (including " + removed[0]->name + ")", ErrorCodes::LOGICAL_ERROR); + if (out_transaction && &out_transaction->data != this) + throw Exception("MergeTreeData::Transaction for one table cannot be used with another. It is a bug.", + ErrorCodes::LOGICAL_ERROR); + + DataPartsVector covered_parts; + { + auto lock = lockParts(); + if (!renameTempPartAndReplace(part, increment, out_transaction, lock, &covered_parts)) + return false; + } + if (!covered_parts.empty()) + throw Exception("Added part " + part->name + " covers " + toString(covered_parts.size()) + + " existing part(s) (including " + covered_parts[0]->name + ")", ErrorCodes::LOGICAL_ERROR); + + return true; } -void MergeTreeData::renameTempPartAndReplace( +bool MergeTreeData::renameTempPartAndReplace( MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction, std::unique_lock & lock, DataPartsVector * out_covered_parts) { @@ -1816,7 +1827,7 @@ void MergeTreeData::renameTempPartAndReplace( if (covering_part) { LOG_WARNING(log, "Tried to add obsolete part {} covered by {}", part_name, covering_part->getNameWithState()); - return; + return false; } /// All checks are passed. Now we can rename the part on disk. @@ -1854,6 +1865,8 @@ void MergeTreeData::renameTempPartAndReplace( for (DataPartPtr & covered_part : covered_parts) out_covered_parts->emplace_back(std::move(covered_part)); } + + return true; } MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace( diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 217e5000cf6..155c2d432c9 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -423,7 +423,8 @@ public: /// If out_transaction != nullptr, adds the part in the PreCommitted state (the part will be added to the /// active set later with out_transaction->commit()). /// Else, commits the part immediately. - void renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr); + /// Returns true if part was added. Returns false if part is covered by bigger part. + bool renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr); /// The same as renameTempPartAndAdd but the block range of the part can contain existing parts. /// Returns all parts covered by the added part (in ascending order). @@ -432,7 +433,7 @@ public: MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr); /// Low-level version of previous one, doesn't lock mutex - void renameTempPartAndReplace( + bool renameTempPartAndReplace( MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction, DataPartsLock & lock, DataPartsVector * out_covered_parts = nullptr); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 1bbc56d940d..03885d90ece 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -27,6 +27,7 @@ namespace ErrorCodes extern const int INSERT_WAS_DEDUPLICATED; extern const int TIMEOUT_EXCEEDED; extern const int NO_ACTIVE_REPLICAS; + extern const int DUPLICATE_DATA_PART; } @@ -204,165 +205,223 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( storage.check(part->getColumns()); assertSessionIsNotExpired(zookeeper); - /// Obtain incremental block number and lock it. The lock holds our intention to add the block to the filesystem. - /// We remove the lock just after renaming the part. In case of exception, block number will be marked as abandoned. - /// Also, make deduplication check. If a duplicate is detected, no nodes are created. + String temporary_part_name = part->name; - /// Allocate new block number and check for duplicates - bool deduplicate_block = !block_id.empty(); - String block_id_path = deduplicate_block ? storage.zookeeper_path + "/blocks/" + block_id : ""; - auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); - - if (!block_number_lock) + while (true) { - LOG_INFO(log, "Block with ID {} already exists; ignoring it.", block_id); - part->is_duplicate = true; - last_block_is_duplicate = true; - ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks); - return; - } + /// Obtain incremental block number and lock it. The lock holds our intention to add the block to the filesystem. + /// We remove the lock just after renaming the part. In case of exception, block number will be marked as abandoned. + /// Also, make deduplication check. If a duplicate is detected, no nodes are created. - Int64 block_number = block_number_lock->getNumber(); + /// Allocate new block number and check for duplicates + bool deduplicate_block = !block_id.empty(); + String block_id_path = deduplicate_block ? storage.zookeeper_path + "/blocks/" + block_id : ""; + auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); - /// Set part attributes according to part_number. Prepare an entry for log. - - part->info.min_block = block_number; - part->info.max_block = block_number; - part->info.level = 0; - - String part_name = part->getNewName(part->info); - part->name = part_name; - - StorageReplicatedMergeTree::LogEntry log_entry; - log_entry.type = StorageReplicatedMergeTree::LogEntry::GET_PART; - log_entry.create_time = time(nullptr); - log_entry.source_replica = storage.replica_name; - log_entry.new_part_name = part_name; - log_entry.quorum = quorum; - log_entry.block_id = block_id; - - /// Simultaneously add information about the part to all the necessary places in ZooKeeper and remove block_number_lock. - - /// Information about the part. - Coordination::Requests ops; - - storage.getCommitPartOps(ops, part, block_id_path); - - /// Replication log. - ops.emplace_back(zkutil::makeCreateRequest( - storage.zookeeper_path + "/log/log-", - log_entry.toString(), - zkutil::CreateMode::PersistentSequential)); - - /// Deletes the information that the block number is used for writing. - block_number_lock->getUnlockOps(ops); - - /** If you need a quorum - create a node in which the quorum is monitored. - * (If such a node already exists, then someone has managed to make another quorum record at the same time, but for it the quorum has not yet been reached. - * You can not do the next quorum record at this time.) - */ - if (quorum) - { - ReplicatedMergeTreeQuorumEntry quorum_entry; - quorum_entry.part_name = part_name; - quorum_entry.required_number_of_replicas = quorum; - quorum_entry.replicas.insert(storage.replica_name); - - /** At this point, this node will contain information that the current replica received a part. - * When other replicas will receive this part (in the usual way, processing the replication log), - * they will add themselves to the contents of this node. - * When it contains information about `quorum` number of replicas, this node is deleted, - * which indicates that the quorum has been reached. - */ - - ops.emplace_back( - zkutil::makeCreateRequest( - quorum_info.status_path, - quorum_entry.toString(), - zkutil::CreateMode::Persistent)); - - /// Make sure that during the insertion time, the replica was not reinitialized or disabled (when the server is finished). - ops.emplace_back( - zkutil::makeCheckRequest( - storage.replica_path + "/is_active", - quorum_info.is_active_node_version)); - - /// Unfortunately, just checking the above is not enough, because `is_active` node can be deleted and reappear with the same version. - /// But then the `host` value will change. We will check this. - /// It's great that these two nodes change in the same transaction (see MergeTreeRestartingThread). - ops.emplace_back( - zkutil::makeCheckRequest( - storage.replica_path + "/host", - quorum_info.host_node_version)); - } - - MergeTreeData::Transaction transaction(storage); /// If you can not add a part to ZK, we'll remove it back from the working set. - storage.renameTempPartAndAdd(part, nullptr, &transaction); - - Coordination::Responses responses; - Coordination::Error multi_code = zookeeper->tryMultiNoThrow(ops, responses); /// 1 RTT - - if (multi_code == Coordination::Error::ZOK) - { - transaction.commit(); - storage.merge_selecting_task->schedule(); - - /// Lock nodes have been already deleted, do not delete them in destructor - block_number_lock->assumeUnlocked(); - } - else if (multi_code == Coordination::Error::ZCONNECTIONLOSS - || multi_code == Coordination::Error::ZOPERATIONTIMEOUT) - { - /** If the connection is lost, and we do not know if the changes were applied, we can not delete the local part - * if the changes were applied, the inserted block appeared in `/blocks/`, and it can not be inserted again. - */ - transaction.commit(); - storage.enqueuePartForCheck(part->name, MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER); - - /// We do not know whether or not data has been inserted. - throw Exception("Unknown status, client must retry. Reason: " + String(Coordination::errorMessage(multi_code)), - ErrorCodes::UNKNOWN_STATUS_OF_INSERT); - } - else if (Coordination::isUserError(multi_code)) - { - String failed_op_path = zkutil::KeeperMultiException(multi_code, ops, responses).getPathForFirstFailedOp(); - - if (multi_code == Coordination::Error::ZNODEEXISTS && deduplicate_block && failed_op_path == block_id_path) + Int64 block_number; + String existing_part_name; + if (block_number_lock) { - /// Block with the same id have just appeared in table (or other replica), rollback thee insertion. - LOG_INFO(log, "Block with ID {} already exists; ignoring it (removing part {})", block_id, part->name); + block_number = block_number_lock->getNumber(); - part->is_duplicate = true; - transaction.rollback(); - last_block_is_duplicate = true; - ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks); - } - else if (multi_code == Coordination::Error::ZNODEEXISTS && failed_op_path == quorum_info.status_path) - { - transaction.rollback(); + /// Set part attributes according to part_number. Prepare an entry for log. - throw Exception("Another quorum insert has been already started", ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE); + part->info.min_block = block_number; + part->info.max_block = block_number; + part->info.level = 0; + + part->name = part->getNewName(part->info); } else { - /// NOTE: We could be here if the node with the quorum existed, but was quickly removed. - transaction.rollback(); - throw Exception("Unexpected logical error while adding block " + toString(block_number) + " with ID '" + block_id + "': " - + Coordination::errorMessage(multi_code) + ", path " + failed_op_path, - ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); + /// This block was already written to some replica. Get the part name for it. + /// Note: race condition with DROP PARTITION operation is possible. User will get "No node" exception and it is Ok. + existing_part_name = zookeeper->get(storage.zookeeper_path + "/blocks/" + block_id); + + /// If it exists on our replica, ignore it. + if (storage.getActiveContainingPart(existing_part_name)) + { + LOG_INFO(log, "Block with ID {} already exists locally as part {}; ignoring it.", block_id, existing_part_name); + part->is_duplicate = true; + last_block_is_duplicate = true; + ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks); + return; + } + + LOG_INFO(log, "Block with ID {} already exists on other replicas as part {}; will write it locally with that name.", + block_id, existing_part_name); + + /// If it does not exist, we will write a new part with existing name. + /// Note that it may also appear on filesystem right now in PreCommitted state due to concurrent inserts of the same data. + /// It will be checked when we will try to rename directory. + + part->name = existing_part_name; + part->info = MergeTreePartInfo::fromPartName(existing_part_name, storage.format_version); + + /// Don't do subsequent duplicate check. + block_id_path.clear(); } - } - else if (Coordination::isHardwareError(multi_code)) - { - transaction.rollback(); - throw Exception("Unrecoverable network error while adding block " + toString(block_number) + " with ID '" + block_id + "': " - + Coordination::errorMessage(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); - } - else - { - transaction.rollback(); - throw Exception("Unexpected ZooKeeper error while adding block " + toString(block_number) + " with ID '" + block_id + "': " - + Coordination::errorMessage(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); + + StorageReplicatedMergeTree::LogEntry log_entry; + log_entry.type = StorageReplicatedMergeTree::LogEntry::GET_PART; + log_entry.create_time = time(nullptr); + log_entry.source_replica = storage.replica_name; + log_entry.new_part_name = part->name; + log_entry.quorum = quorum; + log_entry.block_id = block_id; + + /// Simultaneously add information about the part to all the necessary places in ZooKeeper and remove block_number_lock. + + /// Information about the part. + Coordination::Requests ops; + + storage.getCommitPartOps(ops, part, block_id_path); + + /// Replication log. + ops.emplace_back(zkutil::makeCreateRequest( + storage.zookeeper_path + "/log/log-", + log_entry.toString(), + zkutil::CreateMode::PersistentSequential)); + + /// Deletes the information that the block number is used for writing. + if (block_number_lock) + block_number_lock->getUnlockOps(ops); + + /** If you need a quorum - create a node in which the quorum is monitored. + * (If such a node already exists, then someone has managed to make another quorum record at the same time, + * but for it the quorum has not yet been reached. + * You can not do the next quorum record at this time.) + */ + if (quorum) /// TODO Duplicate blocks. + { + ReplicatedMergeTreeQuorumEntry quorum_entry; + quorum_entry.part_name = part->name; + quorum_entry.required_number_of_replicas = quorum; + quorum_entry.replicas.insert(storage.replica_name); + + /** At this point, this node will contain information that the current replica received a part. + * When other replicas will receive this part (in the usual way, processing the replication log), + * they will add themselves to the contents of this node. + * When it contains information about `quorum` number of replicas, this node is deleted, + * which indicates that the quorum has been reached. + */ + + ops.emplace_back( + zkutil::makeCreateRequest( + quorum_info.status_path, + quorum_entry.toString(), + zkutil::CreateMode::Persistent)); + + /// Make sure that during the insertion time, the replica was not reinitialized or disabled (when the server is finished). + ops.emplace_back( + zkutil::makeCheckRequest( + storage.replica_path + "/is_active", + quorum_info.is_active_node_version)); + + /// Unfortunately, just checking the above is not enough, because `is_active` node can be deleted and reappear with the same version. + /// But then the `host` value will change. We will check this. + /// It's great that these two nodes change in the same transaction (see MergeTreeRestartingThread). + ops.emplace_back( + zkutil::makeCheckRequest( + storage.replica_path + "/host", + quorum_info.host_node_version)); + } + + MergeTreeData::Transaction transaction(storage); /// If you can not add a part to ZK, we'll remove it back from the working set. + bool renamed = false; + try + { + renamed = storage.renameTempPartAndAdd(part, nullptr, &transaction); + } + catch (const Exception & e) + { + if (e.code() != ErrorCodes::DUPLICATE_DATA_PART) + throw; + } + if (!renamed) + { + if (!existing_part_name.empty()) + { + LOG_INFO(log, "Part {} is duplicate and it is already written by concurrent request; ignoring it.", block_id, existing_part_name); + return; + } + else + throw Exception("Part with name {} is already written by concurrent request. It should not happen for non-duplicate data parts because unique names are assigned for them. It's a bug", ErrorCodes::LOGICAL_ERROR); + } + + Coordination::Responses responses; + Coordination::Error multi_code = zookeeper->tryMultiNoThrow(ops, responses); /// 1 RTT + + if (multi_code == Coordination::Error::ZOK) + { + transaction.commit(); + storage.merge_selecting_task->schedule(); + + /// Lock nodes have been already deleted, do not delete them in destructor + if (block_number_lock) + block_number_lock->assumeUnlocked(); + } + else if (multi_code == Coordination::Error::ZCONNECTIONLOSS + || multi_code == Coordination::Error::ZOPERATIONTIMEOUT) + { + /** If the connection is lost, and we do not know if the changes were applied, we can not delete the local part + * if the changes were applied, the inserted block appeared in `/blocks/`, and it can not be inserted again. + */ + transaction.commit(); + storage.enqueuePartForCheck(part->name, MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER); + + /// We do not know whether or not data has been inserted. + throw Exception("Unknown status, client must retry. Reason: " + String(Coordination::errorMessage(multi_code)), + ErrorCodes::UNKNOWN_STATUS_OF_INSERT); + } + else if (Coordination::isUserError(multi_code)) + { + String failed_op_path = zkutil::KeeperMultiException(multi_code, ops, responses).getPathForFirstFailedOp(); + + if (multi_code == Coordination::Error::ZNODEEXISTS && deduplicate_block && failed_op_path == block_id_path) + { + /// Block with the same id have just appeared in table (or other replica), rollback thee insertion. + LOG_INFO(log, "Block with ID {} already exists (it was just appeared). Renaming part {} back to {}. Will retry write.", + block_id, part->name, temporary_part_name); + + transaction.rollback(); + + part->is_duplicate = true; + part->is_temp = true; + part->state = MergeTreeDataPartState::Temporary; + part->renameTo(temporary_part_name); + + continue; + } + else if (multi_code == Coordination::Error::ZNODEEXISTS && failed_op_path == quorum_info.status_path) + { + transaction.rollback(); + + throw Exception("Another quorum insert has been already started", ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE); + } + else + { + /// NOTE: We could be here if the node with the quorum existed, but was quickly removed. + transaction.rollback(); + throw Exception("Unexpected logical error while adding block " + toString(block_number) + " with ID '" + block_id + "': " + + Coordination::errorMessage(multi_code) + ", path " + failed_op_path, + ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); + } + } + else if (Coordination::isHardwareError(multi_code)) + { + transaction.rollback(); + throw Exception("Unrecoverable network error while adding block " + toString(block_number) + " with ID '" + block_id + "': " + + Coordination::errorMessage(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); + } + else + { + transaction.rollback(); + throw Exception("Unexpected ZooKeeper error while adding block " + toString(block_number) + " with ID '" + block_id + "': " + + Coordination::errorMessage(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); + } + + break; } if (quorum) @@ -386,7 +445,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( ReplicatedMergeTreeQuorumEntry quorum_entry(value); /// If the node has time to disappear, and then appear again for the next insert. - if (quorum_entry.part_name != part_name) + if (quorum_entry.part_name != part->name) break; if (!event->tryWait(quorum_timeout_ms)) diff --git a/tests/queries/0_stateless/01319_manual_write_to_replicas.reference b/tests/queries/0_stateless/01319_manual_write_to_replicas.reference new file mode 100644 index 00000000000..0e3a632a4ee --- /dev/null +++ b/tests/queries/0_stateless/01319_manual_write_to_replicas.reference @@ -0,0 +1,6 @@ +Hello, world +--- +Hello, world +Hello, world +Hello, world +Hello, world diff --git a/tests/queries/0_stateless/01319_manual_write_to_replicas.sql b/tests/queries/0_stateless/01319_manual_write_to_replicas.sql new file mode 100644 index 00000000000..5388f0017c0 --- /dev/null +++ b/tests/queries/0_stateless/01319_manual_write_to_replicas.sql @@ -0,0 +1,25 @@ +DROP TABLE IF EXISTS r1; +DROP TABLE IF EXISTS r2; + +CREATE TABLE r1 (x String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/r', 'r1') ORDER BY x; +CREATE TABLE r2 (x String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/r', 'r2') ORDER BY x; + +SYSTEM STOP REPLICATED SENDS; + +INSERT INTO r1 VALUES ('Hello, world'); +SELECT * FROM r1; +SELECT * FROM r2; +INSERT INTO r2 VALUES ('Hello, world'); +SELECT '---'; +SELECT * FROM r1; +SELECT * FROM r2; + +SYSTEM START REPLICATED SENDS; +SYSTEM SYNC REPLICA r1; +SYSTEM SYNC REPLICA r2; + +SELECT * FROM r1; +SELECT * FROM r2; + +DROP TABLE r1; +DROP TABLE r2; From 6da1a0f153f8e6fab578154159594320ba1d6264 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jun 2020 04:13:45 +0300 Subject: [PATCH 0002/1911] Fix style --- src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 03885d90ece..b1d9eeac728 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -28,6 +28,7 @@ namespace ErrorCodes extern const int TIMEOUT_EXCEEDED; extern const int NO_ACTIVE_REPLICAS; extern const int DUPLICATE_DATA_PART; + extern const int LOGICAL_ERROR; } From 4360e326ed86ae62f3f9a016428ba71de34c786f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jun 2020 04:17:02 +0300 Subject: [PATCH 0003/1911] Fix clang-static-analyzer --- .../MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index b1d9eeac728..b78282dad16 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -219,7 +219,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( String block_id_path = deduplicate_block ? storage.zookeeper_path + "/blocks/" + block_id : ""; auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); - Int64 block_number; + Int64 block_number = 0; String existing_part_name; if (block_number_lock) { @@ -259,6 +259,8 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( part->name = existing_part_name; part->info = MergeTreePartInfo::fromPartName(existing_part_name, storage.format_version); + block_number = part->info.min_block; + /// Don't do subsequent duplicate check. block_id_path.clear(); } From 91e78672fa3fb3b74c233516ae85a7ecd89b7c14 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jun 2020 15:19:29 +0300 Subject: [PATCH 0004/1911] Enable compact parts by default for small parts --- src/Interpreters/SystemLog.cpp | 3 +-- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index b432cd8803b..c2e82646de9 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -56,8 +56,7 @@ std::shared_ptr createSystemLog( else { String partition_by = config.getString(config_prefix + ".partition_by", "toYYYYMM(event_date)"); - engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time)" - "SETTINGS min_bytes_for_wide_part = '10M'"; /// Use polymorphic parts for log tables by default + engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time)"; } size_t flush_interval_milliseconds = config.getUInt64(config_prefix + ".flush_interval_milliseconds", DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index f2d2a7cc3d4..d88fe26454c 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -29,7 +29,7 @@ struct MergeTreeSettings : public SettingsCollection M(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \ \ /** Data storing format settings. */ \ - M(SettingUInt64, min_bytes_for_wide_part, 0, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ + M(SettingUInt64, min_bytes_for_wide_part, 10485760, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ M(SettingUInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \ \ /** Merge settings. */ \ From f8f67a788e4c8dc41b59d6f22631172fb4a431df Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 25 Jun 2020 19:55:45 +0300 Subject: [PATCH 0005/1911] allow to turn on fsync on inserts and merges --- src/Disks/DiskLocal.cpp | 17 +++++++- src/Disks/DiskLocal.h | 2 + src/Disks/DiskMemory.cpp | 5 +++ src/Disks/DiskMemory.h | 2 + src/Disks/IDisk.h | 3 ++ src/Disks/S3/DiskS3.cpp | 5 +++ src/Disks/S3/DiskS3.h | 2 + .../MergeTree/IMergeTreeDataPartWriter.cpp | 11 +++-- .../MergeTree/IMergeTreeDataPartWriter.h | 6 +-- .../MergeTree/MergeTreeDataMergerMutator.cpp | 41 +++++++++++++------ .../MergeTree/MergeTreeDataMergerMutator.h | 6 ++- .../MergeTreeDataPartWriterCompact.cpp | 4 +- .../MergeTreeDataPartWriterCompact.h | 2 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 4 +- .../MergeTree/MergeTreeDataPartWriterWide.h | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 7 +++- src/Storages/MergeTree/MergeTreeSettings.h | 3 ++ .../MergeTree/MergedBlockOutputStream.cpp | 7 ++-- .../MergeTree/MergedBlockOutputStream.h | 1 + .../MergedColumnOnlyOutputStream.cpp | 9 ++-- .../MergeTree/MergedColumnOnlyOutputStream.h | 2 +- 21 files changed, 108 insertions(+), 33 deletions(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 68f5ee99a7a..c67bac7ffe2 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -8,7 +8,7 @@ #include #include - +#include namespace DB { @@ -19,6 +19,9 @@ namespace ErrorCodes extern const int EXCESSIVE_ELEMENT_IN_CONFIG; extern const int PATH_ACCESS_DENIED; extern const int INCORRECT_DISK_INDEX; + extern const int FILE_DOESNT_EXIST; + extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_FSYNC; } std::mutex DiskLocal::reservation_mutex; @@ -188,6 +191,18 @@ void DiskLocal::moveDirectory(const String & from_path, const String & to_path) Poco::File(disk_path + from_path).renameTo(disk_path + to_path); } +void DiskLocal::sync(const String & path) const +{ + String full_path = disk_path + path; + int fd = ::open(full_path.c_str(), O_RDONLY); + if (-1 == fd) + throwFromErrnoWithPath("Cannot open file " + full_path, full_path, + errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE); + + if (-1 == fsync(fd)) + throwFromErrnoWithPath("Cannot fsync " + full_path, full_path, ErrorCodes::CANNOT_FSYNC); +} + DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) { return std::make_unique(disk_path, path); diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 61a3994b655..743ba2ceb10 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -59,6 +59,8 @@ public: void moveDirectory(const String & from_path, const String & to_path) override; + void sync(const String & path) const override; + DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void createFile(const String & path) override; diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index 3e43d159ba5..5b3350e40f7 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -261,6 +261,11 @@ void DiskMemory::moveDirectory(const String & /*from_path*/, const String & /*to throw Exception("Method moveDirectory is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); } +void DiskMemory::sync(const String & /*path*/) const +{ + throw Exception("Method sync is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); +} + DiskDirectoryIteratorPtr DiskMemory::iterateDirectory(const String & path) { std::lock_guard lock(mutex); diff --git a/src/Disks/DiskMemory.h b/src/Disks/DiskMemory.h index b0c1d30c61d..8a3ddf05aa7 100644 --- a/src/Disks/DiskMemory.h +++ b/src/Disks/DiskMemory.h @@ -52,6 +52,8 @@ public: void moveDirectory(const String & from_path, const String & to_path) override; + void sync(const String & path) const override; + DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void createFile(const String & path) override; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 011c75402f4..8de77a560d1 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -105,6 +105,9 @@ public: /// Move directory from `from_path` to `to_path`. virtual void moveDirectory(const String & from_path, const String & to_path) = 0; + /// Do fsync on directory. + virtual void sync(const String & path) const = 0; + /// Return iterator to the contents of the specified directory. virtual DiskDirectoryIteratorPtr iterateDirectory(const String & path) = 0; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 71b5991f770..292f6567df4 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -466,6 +466,11 @@ void DiskS3::clearDirectory(const String & path) remove(it->path()); } +void DiskS3::sync(const String & /*path*/) const +{ + throw Exception("Method sync is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); +} + void DiskS3::moveFile(const String & from_path, const String & to_path) { if (exists(to_path)) diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 5fa8e8358a6..09132367ae8 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -58,6 +58,8 @@ public: void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); } + void sync(const String & path) const override; + DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void moveFile(const String & from_path, const String & to_path) override; diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index 73ac7fc0064..03ae2166504 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -308,7 +308,8 @@ void IMergeTreeDataPartWriter::calculateAndSerializeSkipIndices( skip_index_data_mark = skip_index_current_data_mark; } -void IMergeTreeDataPartWriter::finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums) +void IMergeTreeDataPartWriter::finishPrimaryIndexSerialization( + MergeTreeData::DataPart::Checksums & checksums, bool sync) { bool write_final_mark = (with_final_mark && data_written); if (write_final_mark && compute_granularity) @@ -330,12 +331,14 @@ void IMergeTreeDataPartWriter::finishPrimaryIndexSerialization(MergeTreeData::Da index_stream->next(); checksums.files["primary.idx"].file_size = index_stream->count(); checksums.files["primary.idx"].file_hash = index_stream->getHash(); - index_stream = nullptr; + if (sync) + index_stream->sync(); + index_stream.reset(); } } void IMergeTreeDataPartWriter::finishSkipIndicesSerialization( - MergeTreeData::DataPart::Checksums & checksums) + MergeTreeData::DataPart::Checksums & checksums, bool sync) { for (size_t i = 0; i < skip_indices.size(); ++i) { @@ -348,6 +351,8 @@ void IMergeTreeDataPartWriter::finishSkipIndicesSerialization( { stream->finalize(); stream->addToChecksums(checksums); + if (sync) + stream->sync(); } skip_indices_streams.clear(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 2f849e7c895..eebdb880a66 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -102,9 +102,9 @@ public: void initSkipIndices(); void initPrimaryIndex(); - virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) = 0; - void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums); - void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksums); + virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) = 0; + void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums, bool sync); + void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksum, bool sync); protected: /// Count index_granularity for block and store in `index_granularity` diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 00830dd78c2..ccd7f234925 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -576,6 +576,13 @@ public: } }; +static bool needSyncPart(const size_t input_rows, size_t input_bytes, const MergeTreeSettings & settings) +{ + return ((settings.min_rows_to_sync_after_merge && input_rows >= settings.min_rows_to_sync_after_merge) + || (settings.min_compressed_bytes_to_sync_after_merge && input_bytes >= settings.min_compressed_bytes_to_sync_after_merge)); +} + + /// parts should be sorted. MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( const FutureMergedMutatedPart & future_part, @@ -648,6 +655,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor } size_t sum_input_rows_upper_bound = merge_entry->total_rows_count; + size_t sum_compressed_bytes_upper_bound = merge_entry->total_size_bytes_compressed; MergeAlgorithm merge_alg = chooseMergeAlgorithm(parts, sum_input_rows_upper_bound, gathering_columns, deduplicate, need_remove_expired_values); LOG_DEBUG(log, "Selected MergeAlgorithm: {}", ((merge_alg == MergeAlgorithm::Vertical) ? "Vertical" : "Horizontal")); @@ -803,7 +811,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (need_remove_expired_values) merged_stream = std::make_shared(merged_stream, data, metadata_snapshot, new_data_part, time_of_merge, force_ttl); - if (metadata_snapshot->hasSecondaryIndices()) { const auto & indices = metadata_snapshot->getSecondaryIndices(); @@ -863,6 +870,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (need_remove_expired_values && ttl_merges_blocker.isCancelled()) throw Exception("Cancelled merging parts with expired TTL", ErrorCodes::ABORTED); + bool need_sync = needSyncPart(sum_input_rows_upper_bound, sum_compressed_bytes_upper_bound, *data_settings); MergeTreeData::DataPart::Checksums checksums_gathered_columns; /// Gather ordinary columns @@ -942,7 +950,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); column_gathered_stream.readSuffix(); - auto changed_checksums = column_to.writeSuffixAndGetChecksums(new_data_part, checksums_gathered_columns); + auto changed_checksums = column_to.writeSuffixAndGetChecksums(new_data_part, checksums_gathered_columns, need_sync); checksums_gathered_columns.add(std::move(changed_checksums)); if (rows_written != column_elems_written) @@ -979,9 +987,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor } if (merge_alg != MergeAlgorithm::Vertical) - to.writeSuffixAndFinalizePart(new_data_part); + to.writeSuffixAndFinalizePart(new_data_part, need_sync); else - to.writeSuffixAndFinalizePart(new_data_part, &storage_columns, &checksums_gathered_columns); + to.writeSuffixAndFinalizePart(new_data_part, need_sync, &storage_columns, &checksums_gathered_columns); + + if (need_sync) + new_data_part->volume->getDisk()->sync(new_part_tmp_path); return new_data_part; } @@ -1081,7 +1092,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor /// Don't change granularity type while mutating subset of columns auto mrk_extension = source_part->index_granularity_info.is_adaptive ? getAdaptiveMrkExtension(new_data_part->getType()) : getNonAdaptiveMrkExtension(); - + bool need_sync = needSyncPart(source_part->rows_count, source_part->getBytesOnDisk(), *data_settings); bool need_remove_expired_values = false; if (in && shouldExecuteTTL(metadata_snapshot, in->getHeader().getNamesAndTypesList().getNames(), commands_for_part)) @@ -1099,7 +1110,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor time_of_mutation, compression_codec, merge_entry, - need_remove_expired_values); + need_remove_expired_values, + need_sync); /// no finalization required, because mutateAllPartColumns use /// MergedBlockOutputStream which finilaze all part fields itself @@ -1154,7 +1166,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor time_of_mutation, compression_codec, merge_entry, - need_remove_expired_values); + need_remove_expired_values, + need_sync); } for (const auto & [rename_from, rename_to] : files_to_rename) @@ -1174,6 +1187,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor finalizeMutatedPart(source_part, new_data_part, need_remove_expired_values); } + if (need_sync) + new_data_part->volume->getDisk()->sync(new_part_tmp_path); + return new_data_part; } @@ -1599,7 +1615,8 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( time_t time_of_mutation, const CompressionCodecPtr & compression_codec, MergeListEntry & merge_entry, - bool need_remove_expired_values) const + bool need_remove_expired_values, + bool need_sync) const { if (mutating_stream == nullptr) throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR); @@ -1637,7 +1654,7 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( new_data_part->minmax_idx = std::move(minmax_idx); mutating_stream->readSuffix(); - out.writeSuffixAndFinalizePart(new_data_part); + out.writeSuffixAndFinalizePart(new_data_part, need_sync); } void MergeTreeDataMergerMutator::mutateSomePartColumns( @@ -1650,7 +1667,8 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( time_t time_of_mutation, const CompressionCodecPtr & compression_codec, MergeListEntry & merge_entry, - bool need_remove_expired_values) const + bool need_remove_expired_values, + bool need_sync) const { if (mutating_stream == nullptr) throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR); @@ -1684,10 +1702,9 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( mutating_stream->readSuffix(); - auto changed_checksums = out.writeSuffixAndGetChecksums(new_data_part, new_data_part->checksums); + auto changed_checksums = out.writeSuffixAndGetChecksums(new_data_part, new_data_part->checksums, need_sync); new_data_part->checksums.add(std::move(changed_checksums)); - } void MergeTreeDataMergerMutator::finalizeMutatedPart( diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 121cc770d51..23b8d7f681b 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -189,7 +189,8 @@ private: time_t time_of_mutation, const CompressionCodecPtr & codec, MergeListEntry & merge_entry, - bool need_remove_expired_values) const; + bool need_remove_expired_values, + bool need_sync) const; /// Mutate some columns of source part with mutation_stream void mutateSomePartColumns( @@ -202,7 +203,8 @@ private: time_t time_of_mutation, const CompressionCodecPtr & codec, MergeListEntry & merge_entry, - bool need_remove_expired_values) const; + bool need_remove_expired_values, + bool need_sync) const; /// Initialize and write to disk new part fields like checksums, columns, /// etc. diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index f7a3ad75cf5..79800204a3b 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -141,7 +141,7 @@ void MergeTreeDataPartWriterCompact::writeColumnSingleGranule(const ColumnWithTy column.type->serializeBinaryBulkStateSuffix(serialize_settings, state); } -void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) +void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) { if (columns_buffer.size() != 0) writeBlock(header.cloneWithColumns(columns_buffer.releaseColumns())); @@ -158,6 +158,8 @@ void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart: stream->finalize(); stream->addToChecksums(checksums); + if (sync) + stream->sync(); stream.reset(); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 8183c038c4c..dde7deafc58 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -20,7 +20,7 @@ public: void write(const Block & block, const IColumn::Permutation * permutation, const Block & primary_key_block, const Block & skip_indexes_block) override; - void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override; + void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) override; protected: void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index e71ea4d4b94..fcd0249b10c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -264,7 +264,7 @@ void MergeTreeDataPartWriterWide::writeColumn( next_index_offset = current_row - total_rows; } -void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) +void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) { const auto & global_settings = storage.global_context.getSettingsRef(); IDataType::SerializeBinaryBulkSettings serialize_settings; @@ -295,6 +295,8 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Ch { stream.second->finalize(); stream.second->addToChecksums(checksums); + if (sync) + stream.second->sync(); } column_streams.clear(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index f5a9d17f63c..4286065a3ca 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -23,7 +23,7 @@ public: void write(const Block & block, const IColumn::Permutation * permutation, const Block & primary_key_block, const Block & skip_indexes_block) override; - void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override; + void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) override; IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenOffsetColumns & offset_columns); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 099480aca2f..cf8860b7f04 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -303,10 +303,15 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); + bool sync_on_insert = data.getSettings()->sync_after_insert; out.writePrefix(); out.writeWithPermutation(block, perm_ptr); - out.writeSuffixAndFinalizePart(new_data_part); + out.writeSuffixAndFinalizePart(new_data_part, sync_on_insert); + + /// Sync part directory. + if (sync_on_insert) + new_data_part->volume->getDisk()->sync(full_path); ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterRows, block.rows()); ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterUncompressedBytes, block.bytes()); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index f2d2a7cc3d4..da2c9ee49ee 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -43,6 +43,9 @@ struct MergeTreeSettings : public SettingsCollection M(SettingSeconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ M(SettingSeconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ M(SettingSeconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ + M(SettingUInt64, min_rows_to_sync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingUInt64, min_compressed_bytes_to_sync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingBool, sync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ \ /** Inserts settings. */ \ M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index e776a35f21f..5e15084aa7d 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -85,6 +85,7 @@ void MergedBlockOutputStream::writeSuffix() void MergedBlockOutputStream::writeSuffixAndFinalizePart( MergeTreeData::MutableDataPartPtr & new_part, + bool sync, const NamesAndTypesList * total_columns_list, MergeTreeData::DataPart::Checksums * additional_column_checksums) { @@ -95,9 +96,9 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( checksums = std::move(*additional_column_checksums); /// Finish columns serialization. - writer->finishDataSerialization(checksums); - writer->finishPrimaryIndexSerialization(checksums); - writer->finishSkipIndicesSerialization(checksums); + writer->finishDataSerialization(checksums, sync); + writer->finishPrimaryIndexSerialization(checksums, sync); + writer->finishSkipIndicesSerialization(checksums, sync); NamesAndTypesList part_columns; if (!total_columns_list) diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 1a8bf9da822..002ef78a9af 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -46,6 +46,7 @@ public: /// Finilize writing part and fill inner structures void writeSuffixAndFinalizePart( MergeTreeData::MutableDataPartPtr & new_part, + bool sync = false, const NamesAndTypesList * total_columns_list = nullptr, MergeTreeData::DataPart::Checksums * additional_column_checksums = nullptr); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 1faadd0d720..e767fb3f155 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -63,12 +63,15 @@ void MergedColumnOnlyOutputStream::writeSuffix() } MergeTreeData::DataPart::Checksums -MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums) +MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums( + MergeTreeData::MutableDataPartPtr & new_part, + MergeTreeData::DataPart::Checksums & all_checksums, + bool sync) { /// Finish columns serialization. MergeTreeData::DataPart::Checksums checksums; - writer->finishDataSerialization(checksums); - writer->finishSkipIndicesSerialization(checksums); + writer->finishDataSerialization(checksums, sync); + writer->finishSkipIndicesSerialization(checksums, sync); auto columns = new_part->getColumns(); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index 902138ced9d..507a964ede0 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -27,7 +27,7 @@ public: void write(const Block & block) override; void writeSuffix() override; MergeTreeData::DataPart::Checksums - writeSuffixAndGetChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums); + writeSuffixAndGetChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums, bool sync = false); private: Block header; From b2aa565a37076230af2ceaa32ee21fa351d37931 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 27 Jun 2020 00:55:48 +0300 Subject: [PATCH 0006/1911] allow to turn on fsync on inserts, merges and fetches --- src/Common/FileSyncGuard.h | 41 +++++++++++++++++++ src/Disks/DiskLocal.cpp | 35 ++++++++++------ src/Disks/DiskLocal.h | 6 ++- src/Disks/DiskMemory.cpp | 20 ++++++--- src/Disks/DiskMemory.h | 6 ++- src/Disks/IDisk.h | 12 ++++-- src/Disks/S3/DiskS3.cpp | 21 +++++++--- src/Disks/S3/DiskS3.h | 6 ++- src/Storages/MergeTree/DataPartsExchange.cpp | 16 +++++++- src/Storages/MergeTree/DataPartsExchange.h | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 +++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 15 ++++--- .../MergeTree/MergeTreeDataWriter.cpp | 12 +++--- src/Storages/MergeTree/MergeTreeSettings.h | 2 + 14 files changed, 154 insertions(+), 44 deletions(-) create mode 100644 src/Common/FileSyncGuard.h diff --git a/src/Common/FileSyncGuard.h b/src/Common/FileSyncGuard.h new file mode 100644 index 00000000000..5ec9b1d0c98 --- /dev/null +++ b/src/Common/FileSyncGuard.h @@ -0,0 +1,41 @@ +#pragma once + +#include + +namespace DB +{ + +/// Helper class, that recieves file descriptor and does fsync for it in destructor. +/// It's used to keep descriptor open, while doing some operations with it, and do fsync at the end. +/// Guaranties of sequence 'close-reopen-fsync' may depend on kernel version. +/// Source: linux-fsdevel mailing-list https://marc.info/?l=linux-fsdevel&m=152535409207496 +class FileSyncGuard +{ +public: + /// NOTE: If you have already opened descriptor, it's preffered to use + /// this constructor instead of construnctor with path. + FileSyncGuard(const DiskPtr & disk_, int fd_) : disk(disk_), fd(fd_) {} + + FileSyncGuard(const DiskPtr & disk_, const String & path) + : disk(disk_), fd(disk_->open(path, O_RDONLY)) {} + + ~FileSyncGuard() + { + try + { + disk->sync(fd); + disk->close(fd); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + +private: + DiskPtr disk; + int fd = -1; +}; + +} + diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index c67bac7ffe2..f85b69baf5e 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes extern const int FILE_DOESNT_EXIST; extern const int CANNOT_OPEN_FILE; extern const int CANNOT_FSYNC; + extern const int CANNOT_CLOSE_FILE; } std::mutex DiskLocal::reservation_mutex; @@ -191,18 +192,6 @@ void DiskLocal::moveDirectory(const String & from_path, const String & to_path) Poco::File(disk_path + from_path).renameTo(disk_path + to_path); } -void DiskLocal::sync(const String & path) const -{ - String full_path = disk_path + path; - int fd = ::open(full_path.c_str(), O_RDONLY); - if (-1 == fd) - throwFromErrnoWithPath("Cannot open file " + full_path, full_path, - errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE); - - if (-1 == fsync(fd)) - throwFromErrnoWithPath("Cannot fsync " + full_path, full_path, ErrorCodes::CANNOT_FSYNC); -} - DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) { return std::make_unique(disk_path, path); @@ -299,6 +288,28 @@ void DiskLocal::copy(const String & from_path, const std::shared_ptr & to IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } +int DiskLocal::open(const String & path, mode_t mode) const +{ + String full_path = disk_path + path; + int fd = ::open(full_path.c_str(), mode); + if (-1 == fd) + throwFromErrnoWithPath("Cannot open file " + full_path, full_path, + errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE); + return fd; +} + +void DiskLocal::close(int fd) const +{ + if (-1 == ::close(fd)) + throw Exception("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE); +} + +void DiskLocal::sync(int fd) const +{ + if (-1 == ::fsync(fd)) + throw Exception("Cannot fsync", ErrorCodes::CANNOT_FSYNC); +} + DiskPtr DiskLocalReservation::getDisk(size_t i) const { if (i != 0) diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 743ba2ceb10..d70ac06c18b 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -59,8 +59,6 @@ public: void moveDirectory(const String & from_path, const String & to_path) override; - void sync(const String & path) const override; - DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void createFile(const String & path) override; @@ -101,6 +99,10 @@ public: void createHardLink(const String & src_path, const String & dst_path) override; + int open(const String & path, mode_t mode) const override; + void close(int fd) const override; + void sync(int fd) const override; + private: bool tryReserve(UInt64 bytes); diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index 5b3350e40f7..a7f1df04e1f 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -261,11 +261,6 @@ void DiskMemory::moveDirectory(const String & /*from_path*/, const String & /*to throw Exception("Method moveDirectory is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); } -void DiskMemory::sync(const String & /*path*/) const -{ - throw Exception("Method sync is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); -} - DiskDirectoryIteratorPtr DiskMemory::iterateDirectory(const String & path) { std::lock_guard lock(mutex); @@ -413,6 +408,21 @@ void DiskMemory::setReadOnly(const String &) throw Exception("Method setReadOnly is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); } +int DiskMemory::open(const String & /*path*/, mode_t /*mode*/) const +{ + throw Exception("Method open is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); +} + +void DiskMemory::close(int /*fd*/) const +{ + throw Exception("Method close is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); +} + +void DiskMemory::sync(int /*fd*/) const +{ + throw Exception("Method sync is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); +} + using DiskMemoryPtr = std::shared_ptr; diff --git a/src/Disks/DiskMemory.h b/src/Disks/DiskMemory.h index 8a3ddf05aa7..7f111fe5e7d 100644 --- a/src/Disks/DiskMemory.h +++ b/src/Disks/DiskMemory.h @@ -52,8 +52,6 @@ public: void moveDirectory(const String & from_path, const String & to_path) override; - void sync(const String & path) const override; - DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void createFile(const String & path) override; @@ -92,6 +90,10 @@ public: void createHardLink(const String & src_path, const String & dst_path) override; + int open(const String & path, mode_t mode) const override; + void close(int fd) const override; + void sync(int fd) const override; + private: void createDirectoriesImpl(const String & path); void replaceFileImpl(const String & from_path, const String & to_path); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 8de77a560d1..bc5c9381643 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -105,9 +105,6 @@ public: /// Move directory from `from_path` to `to_path`. virtual void moveDirectory(const String & from_path, const String & to_path) = 0; - /// Do fsync on directory. - virtual void sync(const String & path) const = 0; - /// Return iterator to the contents of the specified directory. virtual DiskDirectoryIteratorPtr iterateDirectory(const String & path) = 0; @@ -174,6 +171,15 @@ public: /// Create hardlink from `src_path` to `dst_path`. virtual void createHardLink(const String & src_path, const String & dst_path) = 0; + + /// Wrapper for POSIX open + virtual int open(const String & path, mode_t mode) const = 0; + + /// Wrapper for POSIX close + virtual void close(int fd) const = 0; + + /// Wrapper for POSIX fsync + virtual void sync(int fd) const = 0; }; using DiskPtr = std::shared_ptr; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 292f6567df4..3e0fb05ed6f 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -29,6 +29,7 @@ namespace ErrorCodes extern const int CANNOT_SEEK_THROUGH_FILE; extern const int UNKNOWN_FORMAT; extern const int INCORRECT_DISK_INDEX; + extern const int NOT_IMPLEMENTED; } namespace @@ -466,11 +467,6 @@ void DiskS3::clearDirectory(const String & path) remove(it->path()); } -void DiskS3::sync(const String & /*path*/) const -{ - throw Exception("Method sync is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); -} - void DiskS3::moveFile(const String & from_path, const String & to_path) { if (exists(to_path)) @@ -669,6 +665,21 @@ void DiskS3::setReadOnly(const String & path) Poco::File(metadata_path + path).setReadOnly(true); } +int DiskS3::open(const String & /*path*/, mode_t /*mode*/) const +{ + throw Exception("Method open is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); +} + +void DiskS3::close(int /*fd*/) const +{ + throw Exception("Method close is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); +} + +void DiskS3::sync(int /*fd*/) const +{ + throw Exception("Method sync is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); +} + DiskS3Reservation::~DiskS3Reservation() { try diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 09132367ae8..cbf161da561 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -58,8 +58,6 @@ public: void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); } - void sync(const String & path) const override; - DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void moveFile(const String & from_path, const String & to_path) override; @@ -98,6 +96,10 @@ public: void setReadOnly(const String & path) override; + int open(const String & path, mode_t mode) const override; + void close(int fd) const override; + void sync(int fd) const override; + private: bool tryReserve(UInt64 bytes); diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 6796e630ff2..e7bb8206cd9 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -224,9 +225,9 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( int server_protocol_version = parse(in.getResponseCookie("server_protocol_version", "0")); ReservationPtr reservation; + size_t sum_files_size = 0; if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE) { - size_t sum_files_size; readBinary(sum_files_size, in); if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS) { @@ -247,7 +248,10 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( reservation = data.makeEmptyReservationOnLargestDisk(); } - return downloadPart(part_name, replica_path, to_detached, tmp_prefix_, std::move(reservation), in); + bool sync = (data_settings->min_compressed_bytes_to_sync_after_fetch + && sum_files_size >= data_settings->min_compressed_bytes_to_sync_after_fetch); + + return downloadPart(part_name, replica_path, to_detached, tmp_prefix_, sync, std::move(reservation), in); } MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( @@ -255,6 +259,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( const String & replica_path, bool to_detached, const String & tmp_prefix_, + bool sync, const ReservationPtr reservation, PooledReadWriteBufferFromHTTP & in) { @@ -276,6 +281,10 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( disk->createDirectories(part_download_path); + std::optional sync_guard; + if (data.getSettings()->sync_part_directory) + sync_guard.emplace(disk, part_download_path); + MergeTreeData::DataPart::Checksums checksums; for (size_t i = 0; i < files; ++i) { @@ -316,6 +325,9 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( if (file_name != "checksums.txt" && file_name != "columns.txt") checksums.addFile(file_name, file_size, expected_hash); + + if (sync) + hashing_out.sync(); } assertEOF(in); diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index c1aff6bdba5..e983d6deecf 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -71,6 +71,7 @@ private: const String & replica_path, bool to_detached, const String & tmp_prefix_, + bool sync, const ReservationPtr reservation, PooledReadWriteBufferFromHTTP & in); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 61dfeed6b7c..ab9bb7879aa 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -664,6 +665,10 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ String from = getFullRelativePath(); String to = storage.relative_data_path + new_relative_path + "/"; + std::optional sync_guard; + if (storage.getSettings()->sync_part_directory) + sync_guard.emplace(volume->getDisk(), to); + if (!volume->getDisk()->exists(from)) throw Exception("Part directory " + fullPath(volume->getDisk(), from) + " doesn't exist. Most likely it is logical error.", ErrorCodes::FILE_DOESNT_EXIST); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index ccd7f234925..9c8c4e3c1d5 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -695,6 +696,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor gathering_column_names.clear(); } + std::optional sync_guard; + if (data.getSettings()->sync_part_directory) + sync_guard.emplace(disk, new_part_tmp_path); + /** Read from all parts, merge and write into a new one. * In passing, we calculate expression for sorting. */ @@ -991,9 +996,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor else to.writeSuffixAndFinalizePart(new_data_part, need_sync, &storage_columns, &checksums_gathered_columns); - if (need_sync) - new_data_part->volume->getDisk()->sync(new_part_tmp_path); - return new_data_part; } @@ -1089,6 +1091,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor disk->createDirectories(new_part_tmp_path); + std::optional sync_guard; + if (data.getSettings()->sync_part_directory) + sync_guard.emplace(disk, new_part_tmp_path); + /// Don't change granularity type while mutating subset of columns auto mrk_extension = source_part->index_granularity_info.is_adaptive ? getAdaptiveMrkExtension(new_data_part->getType()) : getNonAdaptiveMrkExtension(); @@ -1187,9 +1193,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor finalizeMutatedPart(source_part, new_data_part, need_remove_expired_values); } - if (need_sync) - new_data_part->volume->getDisk()->sync(new_part_tmp_path); - return new_data_part; } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index cf8860b7f04..01f0b086cea 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace ProfileEvents @@ -259,7 +260,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa new_data_part->volume->getDisk()->removeRecursive(full_path); } - new_data_part->volume->getDisk()->createDirectories(full_path); + const auto disk = new_data_part->volume->getDisk(); + disk->createDirectories(full_path); + + std::optional sync_guard; + if (data.getSettings()->sync_part_directory) + sync_guard.emplace(disk, full_path); /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) @@ -309,10 +315,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa out.writeWithPermutation(block, perm_ptr); out.writeSuffixAndFinalizePart(new_data_part, sync_on_insert); - /// Sync part directory. - if (sync_on_insert) - new_data_part->volume->getDisk()->sync(full_path); - ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterRows, block.rows()); ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterUncompressedBytes, block.bytes()); ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterCompressedBytes, new_data_part->getBytesOnDisk()); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index da2c9ee49ee..c559ce2804e 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -45,7 +45,9 @@ struct MergeTreeSettings : public SettingsCollection M(SettingSeconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ M(SettingUInt64, min_rows_to_sync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ M(SettingUInt64, min_compressed_bytes_to_sync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingUInt64, min_compressed_bytes_to_sync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ M(SettingBool, sync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ + M(SettingBool, sync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ \ /** Inserts settings. */ \ M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ From c5f46b37e6a044c3a690678c7542f749a265a546 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jun 2020 00:18:27 +0300 Subject: [PATCH 0007/1911] Update some tests but not all --- ...system_columns_and_system_tables.reference | 2 +- .../00804_test_delta_codec_compression.sql | 12 ++++----- ...ndex_granularity_collapsing_merge_tree.sql | 18 +++++++------ ..._adaptive_index_granularity_merge_tree.sql | 25 +++++++++++-------- ...index_granularity_replacing_merge_tree.sql | 10 ++++---- ...larity_versioned_collapsing_merge_tree.sql | 6 ++--- ...test_fix_extra_seek_on_compressed_cache.sh | 2 +- .../queries/0_stateless/00933_ttl_simple.sql | 10 +------- ...hecksums_in_system_parts_columns_table.sql | 12 ++++----- .../0_stateless/01039_mergetree_exec_time.sql | 2 +- ...1042_check_query_and_last_granule_size.sql | 4 +-- .../01045_order_by_pk_special_storages.sh | 12 ++++----- .../0_stateless/01055_compact_parts.sql | 3 ++- .../00152_insert_different_granularity.sql | 4 +-- 14 files changed, 61 insertions(+), 61 deletions(-) diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference b/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference index 1d1177ba447..ff02b0ba702 100644 --- a/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference +++ b/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference @@ -6,7 +6,7 @@ │ name2 │ 1 │ 0 │ 0 │ 0 │ │ name3 │ 0 │ 0 │ 0 │ 0 │ └───────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘ -231 1 +147 1 ┌─name────────────────┬─partition_key─┬─sorting_key───┬─primary_key─┬─sampling_key─┐ │ check_system_tables │ date │ date, version │ date │ │ └─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘ diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql index ad104eff92c..91bc45df63d 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql @@ -7,12 +7,12 @@ DROP TABLE IF EXISTS default_codec_synthetic; CREATE TABLE delta_codec_synthetic ( id UInt64 Codec(Delta, ZSTD(3)) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; CREATE TABLE default_codec_synthetic ( id UInt64 Codec(ZSTD(3)) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO delta_codec_synthetic SELECT number FROM system.numbers LIMIT 5000000; INSERT INTO default_codec_synthetic SELECT number FROM system.numbers LIMIT 5000000; @@ -45,12 +45,12 @@ DROP TABLE IF EXISTS default_codec_float; CREATE TABLE delta_codec_float ( id Float64 Codec(Delta, LZ4HC) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; CREATE TABLE default_codec_float ( id Float64 Codec(LZ4HC) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO delta_codec_float SELECT number FROM numbers(1547510400, 500000) WHERE number % 3 == 0 OR number % 5 == 0 OR number % 7 == 0 OR number % 11 == 0; INSERT INTO default_codec_float SELECT * from delta_codec_float; @@ -83,12 +83,12 @@ DROP TABLE IF EXISTS default_codec_string; CREATE TABLE delta_codec_string ( id Float64 Codec(Delta, LZ4) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; CREATE TABLE default_codec_string ( id Float64 Codec(LZ4) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO delta_codec_string SELECT concat(toString(number), toString(number % 100)) FROM numbers(1547510400, 500000); INSERT INTO default_codec_string SELECT * from delta_codec_string; diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql index b610d605e23..5603b722513 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql @@ -8,10 +8,11 @@ CREATE TABLE zero_rows_per_granule ( v2 Int64, Sign Int8 ) ENGINE CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(p) ORDER BY k - SETTINGS index_granularity_bytes=20, write_final_mark = 0, - enable_vertical_merge_algorithm=1, - vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + SETTINGS index_granularity_bytes=20, write_final_mark = 0, + min_bytes_for_wide_part = 0, + enable_vertical_merge_algorithm=1, + vertical_merge_algorithm_min_rows_to_activate=0, + vertical_merge_algorithm_min_columns_to_activate=0; INSERT INTO zero_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, 1), ('2018-05-16', 2, 3000, 4000, 1), ('2018-05-17', 3, 5000, 6000, 1), ('2018-05-18', 4, 7000, 8000, 1); @@ -39,10 +40,11 @@ CREATE TABLE four_rows_per_granule ( v2 Int64, Sign Int8 ) ENGINE CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(p) ORDER BY k - SETTINGS index_granularity_bytes=110, write_final_mark = 0, - enable_vertical_merge_algorithm=1, - vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + SETTINGS index_granularity_bytes=110, write_final_mark = 0, + min_bytes_for_wide_part = 0, + enable_vertical_merge_algorithm=1, + vertical_merge_algorithm_min_rows_to_activate=0, + vertical_merge_algorithm_min_columns_to_activate=0; INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, 1), ('2018-05-16', 2, 3000, 4000, 1), ('2018-05-17', 3, 5000, 6000, 1), ('2018-05-18', 4, 7000, 8000, 1); diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.sql index 18ebebb316c..48b6fae19fe 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.sql @@ -7,7 +7,7 @@ CREATE TABLE zero_rows_per_granule ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, write_final_mark = 0; +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO zero_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -34,7 +34,7 @@ CREATE TABLE two_rows_per_granule ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 40, write_final_mark = 0; +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 40, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO two_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -61,7 +61,7 @@ CREATE TABLE four_rows_per_granule ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0; +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO four_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -95,7 +95,7 @@ CREATE TABLE huge_granularity_small_blocks ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 1000000, write_final_mark = 0; +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 1000000, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO huge_granularity_small_blocks (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -126,7 +126,7 @@ CREATE TABLE adaptive_granularity_alter ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0; +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO adaptive_granularity_alter (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -183,7 +183,8 @@ CREATE TABLE zero_rows_per_granule ( SETTINGS index_granularity_bytes=20, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0; INSERT INTO zero_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -215,7 +216,8 @@ CREATE TABLE two_rows_per_granule ( SETTINGS index_granularity_bytes=40, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0; INSERT INTO two_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -246,7 +248,8 @@ CREATE TABLE four_rows_per_granule ( SETTINGS index_granularity_bytes = 110, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0; INSERT INTO four_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -284,7 +287,8 @@ CREATE TABLE huge_granularity_small_blocks ( SETTINGS index_granularity_bytes=1000000, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0; INSERT INTO huge_granularity_small_blocks (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -319,7 +323,8 @@ CREATE TABLE adaptive_granularity_alter ( SETTINGS index_granularity_bytes=110, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0; INSERT INTO adaptive_granularity_alter (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_replacing_merge_tree.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_replacing_merge_tree.sql index f72d5f0f9cb..53a546f9d0f 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_replacing_merge_tree.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_replacing_merge_tree.sql @@ -10,7 +10,7 @@ CREATE TABLE zero_rows_per_granule ( SETTINGS index_granularity_bytes=20, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO zero_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -41,7 +41,7 @@ CREATE TABLE two_rows_per_granule ( SETTINGS index_granularity_bytes=40, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO two_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -72,7 +72,7 @@ CREATE TABLE four_rows_per_granule ( SETTINGS index_granularity_bytes = 110, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO four_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -118,7 +118,7 @@ CREATE TABLE huge_granularity_small_blocks ( SETTINGS index_granularity_bytes=1000000, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO huge_granularity_small_blocks (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -154,7 +154,7 @@ CREATE TABLE adaptive_granularity_alter ( SETTINGS index_granularity_bytes=110, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO adaptive_granularity_alter (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.sql index c5b65839b2a..05f4dc835e5 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.sql @@ -12,7 +12,7 @@ CREATE TABLE zero_rows_per_granule ( SETTINGS index_granularity_bytes=20, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO zero_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 2, 3000, 4000, 1, 1), ('2018-05-17', 3, 5000, 6000, 1, 1), ('2018-05-18', 4, 7000, 8000, 1, 1); @@ -44,7 +44,7 @@ CREATE TABLE four_rows_per_granule ( SETTINGS index_granularity_bytes=120, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 2, 3000, 4000, 1, 1), ('2018-05-17', 3, 5000, 6000, 1, 1), ('2018-05-18', 4, 7000, 8000, 1, 1); @@ -89,7 +89,7 @@ CREATE TABLE six_rows_per_granule ( SETTINGS index_granularity_bytes=170, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO six_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 1, 1000, 2000, -1, 2); diff --git a/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh b/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh index 1f7571a2404..e0225f0d31d 100755 --- a/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh +++ b/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS small_table" -$CLICKHOUSE_CLIENT --query="CREATE TABLE small_table (a UInt64 default 0, n UInt64) ENGINE = MergeTree() PARTITION BY tuple() ORDER BY (a);" +$CLICKHOUSE_CLIENT --query="CREATE TABLE small_table (a UInt64 default 0, n UInt64) ENGINE = MergeTree() PARTITION BY tuple() ORDER BY (a) SETTINGS min_bytes_for_wide_part = 0;" $CLICKHOUSE_CLIENT --query="INSERT INTO small_table(n) SELECT * from system.numbers limit 100000;" diff --git a/tests/queries/0_stateless/00933_ttl_simple.sql b/tests/queries/0_stateless/00933_ttl_simple.sql index b924faad3f5..83d9962043d 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.sql +++ b/tests/queries/0_stateless/00933_ttl_simple.sql @@ -13,9 +13,8 @@ create table ttl_00933_1 (d DateTime, a Int, b Int) engine = MergeTree order by insert into ttl_00933_1 values (now(), 1, 2); insert into ttl_00933_1 values (now(), 3, 4); insert into ttl_00933_1 values (now() + 1000, 5, 6); +select sleep(1.1) format Null; optimize table ttl_00933_1 final; -- check ttl merge for part with both expired and unexpired values -select sleep(1.1) format Null; -- wait if very fast merge happen -optimize table ttl_00933_1 final; select a, b from ttl_00933_1; drop table if exists ttl_00933_1; @@ -24,7 +23,6 @@ create table ttl_00933_1 (d DateTime, a Int ttl d + interval 1 DAY) engine = Mer insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 3); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1 order by d; @@ -34,7 +32,6 @@ create table ttl_00933_1 (d DateTime, a Int) engine = MergeTree order by tuple() insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl_00933_1 values (toDateTime('2100-10-10 00:00:00'), 3); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1 order by d; @@ -43,7 +40,6 @@ drop table if exists ttl_00933_1; create table ttl_00933_1 (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) ttl d + interval 1 day; insert into ttl_00933_1 values (toDate('2000-10-10'), 1); insert into ttl_00933_1 values (toDate('2100-10-10'), 2); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1 order by d; @@ -52,7 +48,6 @@ drop table if exists ttl_00933_1; create table ttl_00933_1 (b Int, a Int ttl now()-1000) engine = MergeTree order by tuple() partition by tuple(); show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1; @@ -61,7 +56,6 @@ drop table if exists ttl_00933_1; create table ttl_00933_1 (b Int, a Int ttl now()+1000) engine = MergeTree order by tuple() partition by tuple(); show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1; @@ -70,7 +64,6 @@ drop table if exists ttl_00933_1; create table ttl_00933_1 (b Int, a Int ttl today()-1) engine = MergeTree order by tuple() partition by tuple(); show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1; @@ -79,7 +72,6 @@ drop table if exists ttl_00933_1; create table ttl_00933_1 (b Int, a Int ttl today()+1) engine = MergeTree order by tuple() partition by tuple(); show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1; diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql index b9eed1e8183..e865ed609be 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql @@ -1,14 +1,14 @@ DROP TABLE IF EXISTS test_00961; -CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree(d, (a, b), 111); +CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree(d, (a, b), 111) SETTINGS min_bytes_for_wide_part = 0; INSERT INTO test_00961 VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789); -SELECT - name, - table, - hash_of_all_files, - hash_of_uncompressed_files, +SELECT + name, + table, + hash_of_all_files, + hash_of_uncompressed_files, uncompressed_hash_of_compressed_files FROM system.parts WHERE table = 'test_00961' and database = currentDatabase(); diff --git a/tests/queries/0_stateless/01039_mergetree_exec_time.sql b/tests/queries/0_stateless/01039_mergetree_exec_time.sql index 4cefb2e9305..d3aade41cea 100644 --- a/tests/queries/0_stateless/01039_mergetree_exec_time.sql +++ b/tests/queries/0_stateless/01039_mergetree_exec_time.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS tab; -create table tab (A Int64) Engine=MergeTree order by tuple(); +create table tab (A Int64) Engine=MergeTree order by tuple() SETTINGS min_bytes_for_wide_part = 0; insert into tab select cityHash64(number) from numbers(1000); select sum(sleep(0.1)) from tab settings max_block_size = 1, max_execution_time=1; -- { serverError 159 } DROP TABLE IF EXISTS tab; diff --git a/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql b/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql index 9777ea1dc45..c62fe25a041 100644 --- a/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql +++ b/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql @@ -1,7 +1,7 @@ SET check_query_single_value_result = 0; DROP TABLE IF EXISTS check_query_test; -CREATE TABLE check_query_test (SomeKey UInt64, SomeValue String) ENGINE = MergeTree() ORDER BY SomeKey; +CREATE TABLE check_query_test (SomeKey UInt64, SomeValue String) ENGINE = MergeTree() ORDER BY SomeKey SETTINGS min_bytes_for_wide_part = 0; -- Number of rows in last granule should be equals to granularity. -- Rows in this table are short, so granularity will be 8192. @@ -17,7 +17,7 @@ DROP TABLE IF EXISTS check_query_test; DROP TABLE IF EXISTS check_query_test_non_adaptive; -CREATE TABLE check_query_test_non_adaptive (SomeKey UInt64, SomeValue String) ENGINE = MergeTree() ORDER BY SomeKey SETTINGS index_granularity_bytes = 0; +CREATE TABLE check_query_test_non_adaptive (SomeKey UInt64, SomeValue String) ENGINE = MergeTree() ORDER BY SomeKey SETTINGS index_granularity_bytes = 0, min_bytes_for_wide_part = 0; INSERT INTO check_query_test_non_adaptive SELECT number, toString(number) FROM system.numbers LIMIT 81920; diff --git a/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh b/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh index 0898fec802c..b0d63b75dd5 100755 --- a/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh +++ b/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh @@ -10,11 +10,11 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS s2" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS m" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS buf" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mv" - -$CLICKHOUSE_CLIENT -q "CREATE TABLE s1 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3" -$CLICKHOUSE_CLIENT -q "CREATE TABLE s2 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3" -$CLICKHOUSE_CLIENT -q "CREATE TABLE m (a UInt32, s String) engine = Merge(currentDatabase(), 's[1,2]')" +$CLICKHOUSE_CLIENT -q "CREATE TABLE s1 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3 SETTINGS min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE s2 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3 SETTINGS min_bytes_for_wide_part = 0" + +$CLICKHOUSE_CLIENT -q "CREATE TABLE m (a UInt32, s String) engine = Merge(currentDatabase(), 's[1,2]') SETTINGS min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "INSERT INTO s1 select (number % 20) * 2 as n, toString(number * number) from numbers(100000)" $CLICKHOUSE_CLIENT -q "INSERT INTO s2 select (number % 20) * 2 + 1 as n, toString(number * number * number) from numbers(100000)" @@ -45,7 +45,7 @@ else fi $CLICKHOUSE_CLIENT -q "SELECT '---MaterializedView---'" -$CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW mv (a UInt32, s String) engine = MergeTree ORDER BY s POPULATE AS SELECT a, s FROM s1 WHERE a % 7 = 0" +$CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW mv (a UInt32, s String) engine = MergeTree ORDER BY s SETTINGS min_bytes_for_wide_part = 0 POPULATE AS SELECT a, s FROM s1 WHERE a % 7 = 0" $CLICKHOUSE_CLIENT -q "SELECT a, s FROM mv ORDER BY s LIMIT 10" rows_read=`$CLICKHOUSE_CLIENT -q "SELECT a, s FROM mv ORDER BY s LIMIT 10 FORMAT JSON" --max_threads=1 --max_block_size=20 | grep "rows_read" | sed 's/[^0-9]*//g'` @@ -59,4 +59,4 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS s1" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS s2" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS m" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS buf" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mv" \ No newline at end of file +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mv" diff --git a/tests/queries/0_stateless/01055_compact_parts.sql b/tests/queries/0_stateless/01055_compact_parts.sql index 05b0657ba7e..e99af76439c 100755 --- a/tests/queries/0_stateless/01055_compact_parts.sql +++ b/tests/queries/0_stateless/01055_compact_parts.sql @@ -3,9 +3,10 @@ set mutations_sync = 2; drop table if exists mt_compact; create table mt_compact(a UInt64, b UInt64 DEFAULT a * a, s String, n Nested(x UInt32, y String), lc LowCardinality(String)) -engine = MergeTree +engine = MergeTree order by a partition by a % 10 settings index_granularity = 8, +min_bytes_for_wide_part = 0, min_rows_for_wide_part = 10; insert into mt_compact (a, s, n.y, lc) select number, toString((number * 2132214234 + 5434543) % 2133443), ['a', 'b', 'c'], number % 2 ? 'bar' : 'baz' from numbers(90); diff --git a/tests/queries/1_stateful/00152_insert_different_granularity.sql b/tests/queries/1_stateful/00152_insert_different_granularity.sql index 5ca34bbe48e..7e04aedf2dd 100644 --- a/tests/queries/1_stateful/00152_insert_different_granularity.sql +++ b/tests/queries/1_stateful/00152_insert_different_granularity.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS fixed_granularity_table; -CREATE TABLE fixed_granularity_table (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, index_granularity_bytes=0; -- looks like default table before update +CREATE TABLE fixed_granularity_table (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, index_granularity_bytes=0, min_bytes_for_wide_part = 0; -- looks like default table before update ALTER TABLE fixed_granularity_table REPLACE PARTITION 201403 FROM test.hits; @@ -29,7 +29,7 @@ ALTER TABLE test.hits ATTACH PARTITION 201403; DROP TABLE IF EXISTS hits_copy; -CREATE TABLE hits_copy (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, index_granularity_bytes=0; +CREATE TABLE hits_copy (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, index_granularity_bytes=0, min_bytes_for_wide_part = 0; ALTER TABLE hits_copy REPLACE PARTITION 201403 FROM test.hits; From ca346ea13cd0ad0f02a29d59302584c826b52298 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 3 Jul 2020 02:41:37 +0300 Subject: [PATCH 0008/1911] rename fsync-related settings --- src/Storages/MergeTree/DataPartsExchange.cpp | 6 +++--- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeSettings.h | 10 +++++----- 5 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index e7bb8206cd9..72b478cf587 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -248,8 +248,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( reservation = data.makeEmptyReservationOnLargestDisk(); } - bool sync = (data_settings->min_compressed_bytes_to_sync_after_fetch - && sum_files_size >= data_settings->min_compressed_bytes_to_sync_after_fetch); + bool sync = (data_settings->min_compressed_bytes_to_fsync_after_fetch + && sum_files_size >= data_settings->min_compressed_bytes_to_fsync_after_fetch); return downloadPart(part_name, replica_path, to_detached, tmp_prefix_, sync, std::move(reservation), in); } @@ -282,7 +282,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( disk->createDirectories(part_download_path); std::optional sync_guard; - if (data.getSettings()->sync_part_directory) + if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, part_download_path); MergeTreeData::DataPart::Checksums checksums; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index ab9bb7879aa..3d8cb6b7fc5 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -666,7 +666,7 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ String to = storage.relative_data_path + new_relative_path + "/"; std::optional sync_guard; - if (storage.getSettings()->sync_part_directory) + if (storage.getSettings()->fsync_part_directory) sync_guard.emplace(volume->getDisk(), to); if (!volume->getDisk()->exists(from)) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 9c8c4e3c1d5..c39d1981031 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -579,8 +579,8 @@ public: static bool needSyncPart(const size_t input_rows, size_t input_bytes, const MergeTreeSettings & settings) { - return ((settings.min_rows_to_sync_after_merge && input_rows >= settings.min_rows_to_sync_after_merge) - || (settings.min_compressed_bytes_to_sync_after_merge && input_bytes >= settings.min_compressed_bytes_to_sync_after_merge)); + return ((settings.min_rows_to_fsync_after_merge && input_rows >= settings.min_rows_to_fsync_after_merge) + || (settings.min_compressed_bytes_to_fsync_after_merge && input_bytes >= settings.min_compressed_bytes_to_fsync_after_merge)); } @@ -697,7 +697,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor } std::optional sync_guard; - if (data.getSettings()->sync_part_directory) + if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, new_part_tmp_path); /** Read from all parts, merge and write into a new one. @@ -1092,7 +1092,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor disk->createDirectories(new_part_tmp_path); std::optional sync_guard; - if (data.getSettings()->sync_part_directory) + if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, new_part_tmp_path); /// Don't change granularity type while mutating subset of columns diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 01f0b086cea..23210fc604e 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -264,7 +264,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa disk->createDirectories(full_path); std::optional sync_guard; - if (data.getSettings()->sync_part_directory) + if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, full_path); /// If we need to calculate some columns to sort. @@ -309,7 +309,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); - bool sync_on_insert = data.getSettings()->sync_after_insert; + bool sync_on_insert = data.getSettings()->fsync_after_insert; out.writePrefix(); out.writeWithPermutation(block, perm_ptr); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index c559ce2804e..eeee0c4b1e1 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -43,11 +43,11 @@ struct MergeTreeSettings : public SettingsCollection M(SettingSeconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ M(SettingSeconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ M(SettingSeconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ - M(SettingUInt64, min_rows_to_sync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ - M(SettingUInt64, min_compressed_bytes_to_sync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ - M(SettingUInt64, min_compressed_bytes_to_sync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ - M(SettingBool, sync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ - M(SettingBool, sync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ + M(SettingUInt64, min_rows_to_fsync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingUInt64, min_compressed_bytes_to_fsync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingUInt64, min_compressed_bytes_to_fsync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ + M(SettingBool, fsync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ + M(SettingBool, fsync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ \ /** Inserts settings. */ \ M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ From d6434f61dc7b08072862d4d10ea6fa9da781b6c1 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 7 Jul 2020 03:15:02 +0300 Subject: [PATCH 0009/1911] support codecs in compact parts --- .../MergeTreeDataPartWriterCompact.cpp | 80 +++++++++++++------ .../MergeTreeDataPartWriterCompact.h | 21 ++++- .../01375_compact_parts_codecs.reference | 3 + .../01375_compact_parts_codecs.sql | 31 +++++++ 4 files changed, 109 insertions(+), 26 deletions(-) create mode 100644 tests/queries/0_stateless/01375_compact_parts_codecs.reference create mode 100644 tests/queries/0_stateless/01375_compact_parts_codecs.sql diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index f7a3ad75cf5..696197aa4ca 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -14,19 +14,23 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const MergeTreeIndexGranularity & index_granularity_) : IMergeTreeDataPartWriter( data_part_, columns_list_, metadata_snapshot_, indices_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_) + , plain_file(data_part->volume->getDisk()->writeFile( + part_path + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION, + settings.max_compress_block_size, + WriteMode::Rewrite, + settings.estimated_size, + settings.aio_threshold)) + , plain_hashing(*plain_file) + , marks_file(data_part->volume->getDisk()->writeFile( + part_path + MergeTreeDataPartCompact::DATA_FILE_NAME + marks_file_extension_, + 4096, + WriteMode::Rewrite)) + , marks(*marks_file) { - using DataPart = MergeTreeDataPartCompact; - String data_file_name = DataPart::DATA_FILE_NAME; - - stream = std::make_unique( - data_file_name, - data_part->volume->getDisk(), - part_path + data_file_name, DataPart::DATA_FILE_EXTENSION, - part_path + data_file_name, marks_file_extension, - default_codec, - settings.max_compress_block_size, - settings.estimated_size, - settings.aio_threshold); + const auto & storage_columns = metadata_snapshot->getColumns(); + for (const auto & column : columns_list) + compressed_streams[column.name] = std::make_unique( + plain_hashing, storage_columns.getCodecOrDefault(column.name, default_codec)); } void MergeTreeDataPartWriterCompact::write( @@ -98,14 +102,13 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) for (const auto & column : columns_list) { - /// There could already be enough data to compress into the new block. - if (stream->compressed.offset() >= settings.min_compress_block_size) - stream->compressed.next(); + auto & stream = compressed_streams[column.name]; - writeIntBinary(stream->plain_hashing.count(), stream->marks); - writeIntBinary(stream->compressed.offset(), stream->marks); + writeIntBinary(plain_hashing.count(), marks); + writeIntBinary(UInt64(0), marks); writeColumnSingleGranule(block.getByName(column.name), current_row, rows_to_write); + stream->hashing_buf.next(); } ++from_mark; @@ -120,7 +123,7 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) index_granularity.appendMark(rows_written); } - writeIntBinary(rows_to_write, stream->marks); + writeIntBinary(rows_to_write, marks); } next_index_offset = 0; @@ -132,7 +135,7 @@ void MergeTreeDataPartWriterCompact::writeColumnSingleGranule(const ColumnWithTy IDataType::SerializeBinaryBulkStatePtr state; IDataType::SerializeBinaryBulkSettings serialize_settings; - serialize_settings.getter = [this](IDataType::SubstreamPath) -> WriteBuffer * { return &stream->compressed; }; + serialize_settings.getter = [this, &column](IDataType::SubstreamPath) -> WriteBuffer * { return &compressed_streams.at(column.name)->hashing_buf; }; serialize_settings.position_independent_encoding = true; serialize_settings.low_cardinality_max_dictionary_size = 0; @@ -150,15 +153,15 @@ void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart: { for (size_t i = 0; i < columns_list.size(); ++i) { - writeIntBinary(stream->plain_hashing.count(), stream->marks); - writeIntBinary(stream->compressed.offset(), stream->marks); + writeIntBinary(plain_hashing.count(), marks); + writeIntBinary(UInt64(0), marks); } - writeIntBinary(0ULL, stream->marks); + writeIntBinary(UInt64(0), marks); } - stream->finalize(); - stream->addToChecksums(checksums); - stream.reset(); + plain_file->next(); + marks.next(); + addToChecksums(checksums); } static void fillIndexGranularityImpl( @@ -199,6 +202,33 @@ void MergeTreeDataPartWriterCompact::fillIndexGranularity(size_t index_granulari rows_in_block); } +void MergeTreeDataPartWriterCompact::addToChecksums(MergeTreeDataPartChecksums & checksums) +{ + using uint128 = CityHash_v1_0_2::uint128; + + String data_file_name = MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION; + String marks_file_name = MergeTreeDataPartCompact::DATA_FILE_NAME + marks_file_extension; + + checksums.files[data_file_name].is_compressed = true; + size_t uncompressed_size = 0; + uint128 uncompressed_hash{0, 0}; + + for (const auto & [_, stream] : compressed_streams) + { + uncompressed_size += stream->hashing_buf.count(); + uncompressed_hash = CityHash_v1_0_2::CityHash128WithSeed( + reinterpret_cast(&uncompressed_hash), sizeof(uncompressed_hash), uncompressed_hash); + } + + checksums.files[data_file_name].uncompressed_size = uncompressed_size; + checksums.files[data_file_name].uncompressed_hash = uncompressed_hash; + checksums.files[data_file_name].file_size = plain_hashing.count(); + checksums.files[data_file_name].file_hash = plain_hashing.getHash(); + + checksums.files[marks_file_name].file_size = marks.count(); + checksums.files[marks_file_name].file_hash = marks.getHash(); +} + void MergeTreeDataPartWriterCompact::ColumnsBuffer::add(MutableColumns && columns) { if (accumulated_columns.empty()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 8183c038c4c..a5bfd8a16cc 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -34,7 +34,7 @@ private: void writeBlock(const Block & block); - StreamPtr stream; + void addToChecksums(MergeTreeDataPartChecksums & checksumns); Block header; @@ -53,6 +53,25 @@ private: }; ColumnsBuffer columns_buffer; + + /// compressed -> compressed_buf -> plain_hashing -> plain_file + std::unique_ptr plain_file; + HashingWriteBuffer plain_hashing; + + struct CompressedStream + { + CompressedWriteBuffer compressed_buf; + HashingWriteBuffer hashing_buf; + + CompressedStream(WriteBuffer & buf, const CompressionCodecPtr & codec) + : compressed_buf(buf, codec), hashing_buf(compressed_buf) {} + }; + + std::unordered_map> compressed_streams; + + /// marks -> marks_file + std::unique_ptr marks_file; + HashingWriteBuffer marks; }; } diff --git a/tests/queries/0_stateless/01375_compact_parts_codecs.reference b/tests/queries/0_stateless/01375_compact_parts_codecs.reference new file mode 100644 index 00000000000..982c45a26e3 --- /dev/null +++ b/tests/queries/0_stateless/01375_compact_parts_codecs.reference @@ -0,0 +1,3 @@ +12000 11890 +11965 11890 +5858 11890 diff --git a/tests/queries/0_stateless/01375_compact_parts_codecs.sql b/tests/queries/0_stateless/01375_compact_parts_codecs.sql new file mode 100644 index 00000000000..467745c6fa2 --- /dev/null +++ b/tests/queries/0_stateless/01375_compact_parts_codecs.sql @@ -0,0 +1,31 @@ +DROP TABLE IF EXISTS codecs; + +CREATE TABLE codecs (id UInt32, val UInt32, s String) + ENGINE = MergeTree ORDER BY id + SETTINGS min_rows_for_wide_part = 10000; +INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); +SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) + FROM system.parts + WHERE table = 'codecs' AND database = currentDatabase(); + +DROP TABLE codecs; + +CREATE TABLE codecs (id UInt32 CODEC(NONE), val UInt32 CODEC(NONE), s String CODEC(NONE)) + ENGINE = MergeTree ORDER BY id + SETTINGS min_rows_for_wide_part = 10000; +INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); +SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) + FROM system.parts + WHERE table = 'codecs' AND database = currentDatabase(); + +DROP TABLE codecs; + +CREATE TABLE codecs (id UInt32, val UInt32 CODEC(Delta, ZSTD), s String CODEC(ZSTD)) + ENGINE = MergeTree ORDER BY id + SETTINGS min_rows_for_wide_part = 10000; +INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); +SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) + FROM system.parts + WHERE table = 'codecs' AND database = currentDatabase(); + +DROP TABLE codecs; From 2d43519e038ded3bef0962a3d5ff7a5da7248914 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Jul 2020 02:27:20 +0300 Subject: [PATCH 0010/1911] Update some tests --- .../0_stateless/00160_merge_and_index_in_in.sql | 4 ++-- .../0_stateless/00443_preferred_block_size_bytes.sh | 6 +++--- ...0484_preferred_max_column_in_block_size_bytes.sql | 8 ++++---- .../00653_verification_monotonic_data_load.sh | 12 ++++++------ ...checksums_in_system_parts_columns_table.reference | 2 +- ...00961_checksums_in_system_parts_columns_table.sql | 2 +- .../01045_order_by_pk_special_storages.sh | 2 +- .../0_stateless/01343_min_bytes_to_use_mmap_io.sql | 2 +- .../01344_min_bytes_to_use_mmap_io_index.sql | 2 +- 9 files changed, 20 insertions(+), 20 deletions(-) diff --git a/tests/queries/0_stateless/00160_merge_and_index_in_in.sql b/tests/queries/0_stateless/00160_merge_and_index_in_in.sql index 6e2838afe88..bdab3f7640d 100644 --- a/tests/queries/0_stateless/00160_merge_and_index_in_in.sql +++ b/tests/queries/0_stateless/00160_merge_and_index_in_in.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS mt_00160; DROP TABLE IF EXISTS merge_00160; -CREATE TABLE mt_00160 (d Date DEFAULT toDate('2015-05-01'), x UInt64) ENGINE = MergeTree(d, x, 1); +CREATE TABLE mt_00160 (d Date DEFAULT toDate('2015-05-01'), x UInt64) ENGINE = MergeTree PARTITION BY d ORDER BY x SETTINGS index_granularity = 1, min_bytes_for_wide_part = 0; CREATE TABLE merge_00160 (d Date, x UInt64) ENGINE = Merge(currentDatabase(), '^mt_00160$'); SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0; @@ -14,7 +14,7 @@ SELECT *, b FROM merge_00160 WHERE x IN (12345, 67890) AND NOT ignore(blockSize( DROP TABLE merge_00160; DROP TABLE mt_00160; -CREATE TABLE mt_00160 (d Date DEFAULT toDate('2015-05-01'), x UInt64, y UInt64, z UInt64) ENGINE = MergeTree(d, (x, z), 1); +CREATE TABLE mt_00160 (d Date DEFAULT toDate('2015-05-01'), x UInt64, y UInt64, z UInt64) ENGINE = MergeTree PARTITION BY d ORDER BY (x, z) SETTINGS index_granularity = 1, min_bytes_for_wide_part = 0; INSERT INTO mt_00160 (x, y, z) SELECT number AS x, number + 10 AS y, number / 2 AS z FROM system.numbers LIMIT 100000; diff --git a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh index 4bf104a2d03..c05611783bb 100755 --- a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh +++ b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS preferred_block_size_bytes" -$CLICKHOUSE_CLIENT -q "CREATE TABLE preferred_block_size_bytes (p Date, s String) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=1, index_granularity_bytes=0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE preferred_block_size_bytes (p Date, s String) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=1, index_granularity_bytes=0, min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "INSERT INTO preferred_block_size_bytes (s) SELECT '16_bytes_-_-_-_' AS s FROM system.numbers LIMIT 10, 90" $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE preferred_block_size_bytes" $CLICKHOUSE_CLIENT --preferred_block_size_bytes=26 -q "SELECT DISTINCT blockSize(), ignore(p, s) FROM preferred_block_size_bytes" @@ -17,7 +17,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS preferred_block_size_bytes" # PREWHERE using empty column $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS pbs" -$CLICKHOUSE_CLIENT -q "CREATE TABLE pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=100, index_granularity_bytes=0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=100, index_granularity_bytes=0, min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "INSERT INTO pbs (p, i, sa) SELECT toDate(i % 30) AS p, number AS i, ['a'] AS sa FROM system.numbers LIMIT 1000" $CLICKHOUSE_CLIENT -q "ALTER TABLE pbs ADD COLUMN s UInt8 DEFAULT 0" $CLICKHOUSE_CLIENT --preferred_block_size_bytes=100000 -q "SELECT count() FROM pbs PREWHERE s = 0" @@ -28,7 +28,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE pbs" # Nullable PREWHERE $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS nullable_prewhere" -$CLICKHOUSE_CLIENT -q "CREATE TABLE nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=8, index_granularity_bytes=0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=8, index_granularity_bytes=0, min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "INSERT INTO nullable_prewhere SELECT toDate(0) AS p, if(number % 2 = 0, CAST(number AS Nullable(UInt64)), CAST(NULL AS Nullable(UInt64))) AS f, number as d FROM system.numbers LIMIT 1001" $CLICKHOUSE_CLIENT -q "SELECT sum(d), sum(f), max(d) FROM nullable_prewhere PREWHERE NOT isNull(f)" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS nullable_prewhere" diff --git a/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql b/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql index e3ab4180d4e..dc021ad52db 100644 --- a/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql +++ b/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql @@ -1,5 +1,5 @@ drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree(date, (date, x), 8192); +create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; insert into tab_00484 select today(), number, toFixedString('', 128) from system.numbers limit 8192; set preferred_block_size_bytes = 2000000; @@ -15,19 +15,19 @@ set preferred_max_column_in_block_size_bytes = 4194304; select max(blockSize()), min(blockSize()), any(ignore(*)) from tab_00484; drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree(date, (date, x), 32); +create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; insert into tab_00484 select today(), number, toFixedString('', 128) from system.numbers limit 47; set preferred_max_column_in_block_size_bytes = 1152; select blockSize(), * from tab_00484 where x = 1 or x > 36 format Null; drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree(date, (date, x), 8192); +create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; insert into tab_00484 select today(), number, toFixedString('', 128) from system.numbers limit 10; set preferred_max_column_in_block_size_bytes = 128; select s from tab_00484 where s == '' format Null; drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s String) engine = MergeTree(date, (date, x), 8192); +create table tab_00484 (date Date, x UInt64, s String) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; insert into tab_00484 select today(), number, 'abc' from system.numbers limit 81920; set preferred_block_size_bytes = 0; select count(*) from tab_00484 prewhere s != 'abc' format Null; diff --git a/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh b/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh index e52610f03ba..3a8c2445e24 100755 --- a/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh +++ b/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh @@ -20,12 +20,12 @@ ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS unsigned_integer_test_table;" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS enum_test_table;" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS date_test_table;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE string_test_table (val String) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE fixed_string_test_table (val FixedString(1)) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE signed_integer_test_table (val Int32) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE unsigned_integer_test_table (val UInt32) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE enum_test_table (val Enum16('hello' = 1, 'world' = 2, 'yandex' = 256, 'clickhouse' = 257)) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE date_test_table (val Date) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE string_test_table (val String) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE fixed_string_test_table (val FixedString(1)) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE signed_integer_test_table (val Int32) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE unsigned_integer_test_table (val UInt32) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE enum_test_table (val Enum16('hello' = 1, 'world' = 2, 'yandex' = 256, 'clickhouse' = 257)) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE date_test_table (val Date) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;" ${CLICKHOUSE_CLIENT} --query="SYSTEM STOP MERGES;" diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference index 282b0ddca7b..3bcfc00eded 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference @@ -1 +1 @@ -20000101_20000101_1_1_0 test_00961 1c63ae7a38eb76e2a71c28aaf0b3ae4d 0053df9b467cc5483e752ec62e91cfd4 da96ff1e527a8a1f908ddf2b1d0af239 +20000101_20000101_1_1_0 test_00961 b78f351b7498ecc9d4732ad29c3952de 1d4b7fbf05d0fc5c2f4559ca75aa32f7 38f047b57fd1bb81cf77e273deb34218 diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql index e865ed609be..f3a729dd4fd 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS test_00961; -CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree(d, (a, b), 111) SETTINGS min_bytes_for_wide_part = 0; +CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree(d, (a, b), 111); INSERT INTO test_00961 VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789); diff --git a/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh b/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh index b0d63b75dd5..3c549fa64ff 100755 --- a/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh +++ b/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh @@ -14,7 +14,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mv" $CLICKHOUSE_CLIENT -q "CREATE TABLE s1 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3 SETTINGS min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "CREATE TABLE s2 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3 SETTINGS min_bytes_for_wide_part = 0" -$CLICKHOUSE_CLIENT -q "CREATE TABLE m (a UInt32, s String) engine = Merge(currentDatabase(), 's[1,2]') SETTINGS min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE m (a UInt32, s String) engine = Merge(currentDatabase(), 's[1,2]')" $CLICKHOUSE_CLIENT -q "INSERT INTO s1 select (number % 20) * 2 as n, toString(number * number) from numbers(100000)" $CLICKHOUSE_CLIENT -q "INSERT INTO s2 select (number % 20) * 2 + 1 as n, toString(number * number * number) from numbers(100000)" diff --git a/tests/queries/0_stateless/01343_min_bytes_to_use_mmap_io.sql b/tests/queries/0_stateless/01343_min_bytes_to_use_mmap_io.sql index 9ff16ca60a7..62c5d20d714 100644 --- a/tests/queries/0_stateless/01343_min_bytes_to_use_mmap_io.sql +++ b/tests/queries/0_stateless/01343_min_bytes_to_use_mmap_io.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS test_01343; -CREATE TABLE test_01343 (x String) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE test_01343 (x String) ENGINE = MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO test_01343 VALUES ('Hello, world'); SET min_bytes_to_use_mmap_io = 1; diff --git a/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql b/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql index 67baef7136d..544c0af7925 100644 --- a/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql +++ b/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS test_01344; -CREATE TABLE test_01344 (x String, INDEX idx (x) TYPE set(10) GRANULARITY 1) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE test_01344 (x String, INDEX idx (x) TYPE set(10) GRANULARITY 1) ENGINE = MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO test_01344 VALUES ('Hello, world'); SET min_bytes_to_use_mmap_io = 1; From 80a62977f1aa430144a6bdfae0b7e37605eb5b20 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 9 Jul 2020 21:26:54 +0300 Subject: [PATCH 0011/1911] fix hashing in DataPartWriterCompact --- src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 69e581a6299..ac697e1b212 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -217,8 +217,9 @@ void MergeTreeDataPartWriterCompact::addToChecksums(MergeTreeDataPartChecksums & for (const auto & [_, stream] : compressed_streams) { uncompressed_size += stream->hashing_buf.count(); + auto stream_hash = stream->hashing_buf.getHash(); uncompressed_hash = CityHash_v1_0_2::CityHash128WithSeed( - reinterpret_cast(&uncompressed_hash), sizeof(uncompressed_hash), uncompressed_hash); + reinterpret_cast(&stream_hash), sizeof(stream_hash), uncompressed_hash); } checksums.files[data_file_name].uncompressed_size = uncompressed_size; From 24f627e52c5f6f461cd1bc42b2306725ad0491b8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 10 Jul 2020 18:57:10 +0300 Subject: [PATCH 0012/1911] fix reading from compact parts with different codecs --- .../CachedCompressedReadBuffer.cpp | 14 +++- src/Compression/CachedCompressedReadBuffer.h | 4 +- .../CompressedReadBufferFromFile.cpp | 6 ++ .../CompressedReadBufferFromFile.h | 1 + .../MergeTree/MergeTreeReaderCompact.cpp | 80 ++++++++++--------- .../MergeTree/MergeTreeReaderCompact.h | 21 ++++- 6 files changed, 81 insertions(+), 45 deletions(-) diff --git a/src/Compression/CachedCompressedReadBuffer.cpp b/src/Compression/CachedCompressedReadBuffer.cpp index 1b083c004c0..beb13d15f01 100644 --- a/src/Compression/CachedCompressedReadBuffer.cpp +++ b/src/Compression/CachedCompressedReadBuffer.cpp @@ -12,6 +12,7 @@ namespace DB namespace ErrorCodes { extern const int SEEK_POSITION_OUT_OF_BOUND; + extern const int LOGICAL_ERROR; } @@ -19,8 +20,9 @@ void CachedCompressedReadBuffer::initInput() { if (!file_in) { - file_in = file_in_creator(); - compressed_in = file_in.get(); + file_in_holder = file_in_creator(); + file_in = file_in_holder.get(); + compressed_in = file_in; if (profile_callback) file_in->setProfileCallback(profile_callback, clock_type); @@ -71,6 +73,14 @@ bool CachedCompressedReadBuffer::nextImpl() return true; } +CachedCompressedReadBuffer::CachedCompressedReadBuffer( + const std::string & path_, ReadBufferFromFileBase * file_in_, UncompressedCache * cache_) + : ReadBuffer(nullptr, 0), file_in(file_in_), cache(cache_), path(path_), file_pos(0) +{ + if (file_in == nullptr) + throw Exception("Neither file_in nor file_in_creator is initialized in CachedCompressedReadBuffer", ErrorCodes::LOGICAL_ERROR); +} + CachedCompressedReadBuffer::CachedCompressedReadBuffer( const std::string & path_, std::function()> file_in_creator_, UncompressedCache * cache_) : ReadBuffer(nullptr, 0), file_in_creator(std::move(file_in_creator_)), cache(cache_), path(path_), file_pos(0) diff --git a/src/Compression/CachedCompressedReadBuffer.h b/src/Compression/CachedCompressedReadBuffer.h index 88bcec8197d..2c5aa4920bd 100644 --- a/src/Compression/CachedCompressedReadBuffer.h +++ b/src/Compression/CachedCompressedReadBuffer.h @@ -22,7 +22,8 @@ class CachedCompressedReadBuffer : public CompressedReadBufferBase, public ReadB private: std::function()> file_in_creator; UncompressedCache * cache; - std::unique_ptr file_in; + std::unique_ptr file_in_holder; + ReadBufferFromFileBase * file_in; const std::string path; size_t file_pos; @@ -38,6 +39,7 @@ private: clockid_t clock_type {}; public: + CachedCompressedReadBuffer(const std::string & path_, ReadBufferFromFileBase * file_in_, UncompressedCache * cache_); CachedCompressedReadBuffer(const std::string & path, std::function()> file_in_creator, UncompressedCache * cache_); void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block); diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index ddd8bba686f..2927ee1b399 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -37,6 +37,12 @@ bool CompressedReadBufferFromFile::nextImpl() return true; } +CompressedReadBufferFromFile::CompressedReadBufferFromFile(ReadBufferFromFileBase & file_in_) + : BufferWithOwnMemory(0), file_in(file_in_) +{ + compressed_in = &file_in; +} + CompressedReadBufferFromFile::CompressedReadBufferFromFile(std::unique_ptr buf) : BufferWithOwnMemory(0), p_file_in(std::move(buf)), file_in(*p_file_in) { diff --git a/src/Compression/CompressedReadBufferFromFile.h b/src/Compression/CompressedReadBufferFromFile.h index 1729490f606..1de28062e41 100644 --- a/src/Compression/CompressedReadBufferFromFile.h +++ b/src/Compression/CompressedReadBufferFromFile.h @@ -28,6 +28,7 @@ private: bool nextImpl() override; public: + CompressedReadBufferFromFile(ReadBufferFromFileBase & buf); CompressedReadBufferFromFile(std::unique_ptr buf); CompressedReadBufferFromFile( diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 4357ee66a6e..920f171d7f9 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -45,40 +45,31 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( { size_t buffer_size = settings.max_read_buffer_size; const String full_data_path = data_part->getFullRelativePath() + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION; + file_in = data_part->volume->getDisk()->readFile( + full_data_path, buffer_size, 0, + settings.min_bytes_to_use_direct_io, + settings.min_bytes_to_use_mmap_io); - if (uncompressed_cache) + auto full_path = fullPath(data_part->volume->getDisk(), full_data_path); + for (const auto & column : columns) { - auto buffer = std::make_unique( - fullPath(data_part->volume->getDisk(), full_data_path), - [this, full_data_path, buffer_size]() - { - return data_part->volume->getDisk()->readFile( - full_data_path, - buffer_size, - 0, - settings.min_bytes_to_use_direct_io, - settings.min_bytes_to_use_mmap_io); - }, - uncompressed_cache); + + std::unique_ptr cached_buffer; + std::unique_ptr non_cached_buffer; + if (uncompressed_cache) + { + cached_buffer = std::make_unique(full_path, file_in.get(), uncompressed_cache); + if (profile_callback_) + cached_buffer->setProfileCallback(profile_callback_, clock_type_); + } + else + { + non_cached_buffer = std::make_unique(*file_in); + if (profile_callback_) + non_cached_buffer->setProfileCallback(profile_callback_, clock_type_); + } - if (profile_callback_) - buffer->setProfileCallback(profile_callback_, clock_type_); - - cached_buffer = std::move(buffer); - data_buffer = cached_buffer.get(); - } - else - { - auto buffer = - std::make_unique( - data_part->volume->getDisk()->readFile( - full_data_path, buffer_size, 0, settings.min_bytes_to_use_direct_io, settings.min_bytes_to_use_mmap_io)); - - if (profile_callback_) - buffer->setProfileCallback(profile_callback_, clock_type_); - - non_cached_buffer = std::move(buffer); - data_buffer = non_cached_buffer.get(); + column_streams[column.name] = ColumnStream{std::move(cached_buffer), std::move(non_cached_buffer)}; } size_t columns_num = columns.size(); @@ -181,15 +172,16 @@ void MergeTreeReaderCompact::readData( const String & name, IColumn & column, const IDataType & type, size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets) { + auto & stream = column_streams[name]; if (!isContinuousReading(from_mark, column_position)) - seekToMark(from_mark, column_position); + seekToMark(stream, from_mark, column_position); auto buffer_getter = [&](const IDataType::SubstreamPath & substream_path) -> ReadBuffer * { if (only_offsets && (substream_path.size() != 1 || substream_path[0].type != IDataType::Substream::ArraySizes)) return nullptr; - return data_buffer; + return stream.data_buffer; }; IDataType::DeserializeBinaryBulkSettings deserialize_settings; @@ -209,15 +201,15 @@ void MergeTreeReaderCompact::readData( } -void MergeTreeReaderCompact::seekToMark(size_t row_index, size_t column_index) +void MergeTreeReaderCompact::seekToMark(ColumnStream & stream, size_t row_index, size_t column_index) { MarkInCompressedFile mark = marks_loader.getMark(row_index, column_index); try { - if (cached_buffer) - cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); - if (non_cached_buffer) - non_cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); + if (stream.cached_buffer) + stream.cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); + if (stream.non_cached_buffer) + stream.non_cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); } catch (Exception & e) { @@ -239,4 +231,16 @@ bool MergeTreeReaderCompact::isContinuousReading(size_t mark, size_t column_posi || (mark == last_mark + 1 && column_position == 0 && last_column == data_part->getColumns().size() - 1); } +MergeTreeReaderCompact::ColumnStream::ColumnStream( + std::unique_ptr cached_buffer_, + std::unique_ptr non_cached_buffer_) + : cached_buffer(std::move(cached_buffer_)) + , non_cached_buffer(std::move(non_cached_buffer_)) +{ + if (cached_buffer) + data_buffer = cached_buffer.get(); + else + data_buffer = non_cached_buffer.get(); +} + } diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index 0457b4b6a50..41682f8b0bd 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -35,9 +36,21 @@ public: private: bool isContinuousReading(size_t mark, size_t column_position); - ReadBuffer * data_buffer; - std::unique_ptr cached_buffer; - std::unique_ptr non_cached_buffer; + std::unique_ptr file_in; + + struct ColumnStream + { + std::unique_ptr cached_buffer; + std::unique_ptr non_cached_buffer; + ReadBuffer * data_buffer; + + ColumnStream() = default; + ColumnStream( + std::unique_ptr cached_buffer_, + std::unique_ptr non_cached_buffer_); + }; + + std::unordered_map column_streams; MergeTreeMarksLoader marks_loader; @@ -49,7 +62,7 @@ private: size_t next_mark = 0; std::optional> last_read_granule; - void seekToMark(size_t row_index, size_t column_index); + void seekToMark(ColumnStream & stream, size_t row_index, size_t column_index); void readData(const String & name, IColumn & column, const IDataType & type, size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets = false); From fbec940e0fa7246ca7b42e056de7d0cea50640d0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 11 Jul 2020 02:33:36 +0300 Subject: [PATCH 0013/1911] fix reading and check query for compact parts with different codecs --- .../CachedCompressedReadBuffer.cpp | 2 ++ src/Compression/CachedCompressedReadBuffer.h | 2 +- .../MergeTreeDataPartWriterCompact.cpp | 4 ++-- .../MergeTree/MergeTreeReaderCompact.cpp | 5 +++-- src/Storages/MergeTree/checkDataPart.cpp | 22 ++++++++++++++----- .../01390_check_table_codec.reference | 2 ++ .../0_stateless/01390_check_table_codec.sql | 15 +++++++++++++ 7 files changed, 41 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/01390_check_table_codec.reference create mode 100644 tests/queries/0_stateless/01390_check_table_codec.sql diff --git a/src/Compression/CachedCompressedReadBuffer.cpp b/src/Compression/CachedCompressedReadBuffer.cpp index beb13d15f01..218925f8eae 100644 --- a/src/Compression/CachedCompressedReadBuffer.cpp +++ b/src/Compression/CachedCompressedReadBuffer.cpp @@ -79,6 +79,8 @@ CachedCompressedReadBuffer::CachedCompressedReadBuffer( { if (file_in == nullptr) throw Exception("Neither file_in nor file_in_creator is initialized in CachedCompressedReadBuffer", ErrorCodes::LOGICAL_ERROR); + + compressed_in = file_in; } CachedCompressedReadBuffer::CachedCompressedReadBuffer( diff --git a/src/Compression/CachedCompressedReadBuffer.h b/src/Compression/CachedCompressedReadBuffer.h index 2c5aa4920bd..89bf66a3e2c 100644 --- a/src/Compression/CachedCompressedReadBuffer.h +++ b/src/Compression/CachedCompressedReadBuffer.h @@ -23,7 +23,7 @@ private: std::function()> file_in_creator; UncompressedCache * cache; std::unique_ptr file_in_holder; - ReadBufferFromFileBase * file_in; + ReadBufferFromFileBase * file_in = nullptr; const std::string path; size_t file_pos; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index ac697e1b212..d15bba232d6 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -17,7 +17,7 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( default_codec_, settings_, index_granularity_) , plain_file(data_part->volume->getDisk()->writeFile( part_path + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION, - settings.max_compress_block_size, + settings.max_compress_block_size, WriteMode::Rewrite, settings.estimated_size, settings.aio_threshold)) @@ -31,7 +31,7 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const auto & storage_columns = metadata_snapshot->getColumns(); for (const auto & column : columns_list) compressed_streams[column.name] = std::make_unique( - plain_hashing, storage_columns.getCodecOrDefault(column.name, default_codec)); + plain_hashing, storage_columns.getCodecOrDefault(column.name, default_codec)); } void MergeTreeDataPartWriterCompact::write( diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 920f171d7f9..89ca8b96dba 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -53,7 +53,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( auto full_path = fullPath(data_part->volume->getDisk(), full_data_path); for (const auto & column : columns) { - + std::unique_ptr cached_buffer; std::unique_ptr non_cached_buffer; if (uncompressed_cache) @@ -69,7 +69,8 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( non_cached_buffer->setProfileCallback(profile_callback_, clock_type_); } - column_streams[column.name] = ColumnStream{std::move(cached_buffer), std::move(non_cached_buffer)}; + auto column_from_part = getColumnFromPart(column); + column_streams[column_from_part.name] = ColumnStream{std::move(cached_buffer), std::move(non_cached_buffer)}; } size_t columns_num = columns.size(); diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 793bddc88c0..790a250d831 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -96,11 +96,24 @@ IMergeTreeDataPart::Checksums checkDataPart( }; }; + /// This function calculates only checksum of file content (compressed or uncompressed). + auto checksum_file = [](const DiskPtr & disk_, const String & file_path) + { + auto file_buf = disk_->readFile(file_path); + HashingReadBuffer hashing_buf(*file_buf); + hashing_buf.tryIgnore(std::numeric_limits::max()); + return IMergeTreeDataPart::Checksums::Checksum{hashing_buf.count(), hashing_buf.getHash()}; + }; + + bool check_uncompressed = true; /// First calculate checksums for columns data if (part_type == MergeTreeDataPartType::COMPACT) { const auto & file_name = MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION; - checksums_data.files[file_name] = checksum_compressed_file(disk, path + file_name); + checksums_data.files[file_name] = checksum_file(disk, path + file_name); + /// Uncompressed checksums in compact parts are computed in a complex way. + /// We check only checksum of compressed file. + check_uncompressed = false; } else if (part_type == MergeTreeDataPartType::WIDE) { @@ -141,10 +154,7 @@ IMergeTreeDataPart::Checksums checkDataPart( if (txt_checksum_it == checksum_files_txt.end() || txt_checksum_it->second.uncompressed_size == 0) { /// The file is not compressed. - auto file_buf = disk->readFile(it->path()); - HashingReadBuffer hashing_buf(*file_buf); - hashing_buf.tryIgnore(std::numeric_limits::max()); - checksums_data.files[file_name] = IMergeTreeDataPart::Checksums::Checksum(hashing_buf.count(), hashing_buf.getHash()); + checksums_data.files[file_name] = checksum_file(disk, it->path()); } else /// If we have both compressed and uncompressed in txt, than calculate them { @@ -157,7 +167,7 @@ IMergeTreeDataPart::Checksums checkDataPart( return {}; if (require_checksums || !checksums_txt.files.empty()) - checksums_txt.checkEqual(checksums_data, true); + checksums_txt.checkEqual(checksums_data, check_uncompressed); return checksums_data; } diff --git a/tests/queries/0_stateless/01390_check_table_codec.reference b/tests/queries/0_stateless/01390_check_table_codec.reference new file mode 100644 index 00000000000..3025e6463d8 --- /dev/null +++ b/tests/queries/0_stateless/01390_check_table_codec.reference @@ -0,0 +1,2 @@ +all_1_1_0 1 +all_1_1_0 1 diff --git a/tests/queries/0_stateless/01390_check_table_codec.sql b/tests/queries/0_stateless/01390_check_table_codec.sql new file mode 100644 index 00000000000..639d5bea6e4 --- /dev/null +++ b/tests/queries/0_stateless/01390_check_table_codec.sql @@ -0,0 +1,15 @@ +SET check_query_single_value_result = 0; + +DROP TABLE IF EXISTS check_codec; + +CREATE TABLE check_codec(a Int, b Int CODEC(Delta, ZSTD)) ENGINE = MergeTree ORDER BY a SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO check_codec SELECT number, number * 2 FROM numbers(1000); +CHECK TABLE check_codec; + +DROP TABLE check_codec; + +CREATE TABLE check_codec(a Int, b Int CODEC(Delta, ZSTD)) ENGINE = MergeTree ORDER BY a SETTINGS min_bytes_for_wide_part = '10M'; +INSERT INTO check_codec SELECT number, number * 2 FROM numbers(1000); +CHECK TABLE check_codec; + +DROP TABLE check_codec; From 9384b6950b6c5311202788c8b38ed84dd53a13e8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 14 Jul 2020 15:10:20 +0300 Subject: [PATCH 0014/1911] Add some tests configs --- tests/ci/build_config.json | 153 +++++++++++++++++++++++ tests/ci/tests_config.json | 242 +++++++++++++++++++++++++++++++++++++ 2 files changed, 395 insertions(+) create mode 100644 tests/ci/build_config.json create mode 100644 tests/ci/tests_config.json diff --git a/tests/ci/build_config.json b/tests/ci/build_config.json new file mode 100644 index 00000000000..e4b9c1d6b75 --- /dev/null +++ b/tests/ci/build_config.json @@ -0,0 +1,153 @@ +[ + { + "compiler": "gcc-9", + "build-type": "", + "sanitizer": "", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "alien_pkgs": true, + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "gcc-9", + "build-type": "", + "sanitizer": "", + "package-type": "performance", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "gcc-9", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "address", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "undefined", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "thread", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "memory", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "debug", + "sanitizer": "", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "gcc-9", + "build-type": "", + "sanitizer": "", + "package-type": "deb", + "bundled": "unbundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "splitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10-darwin", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10-aarch64", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10-freebsd", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + } +] diff --git a/tests/ci/tests_config.json b/tests/ci/tests_config.json new file mode 100644 index 00000000000..481de51d08b --- /dev/null +++ b/tests/ci/tests_config.json @@ -0,0 +1,242 @@ +{ + "Functional stateful tests (address)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (thread)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "thread", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (memory)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "memory", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (ubsan)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "undefined", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (debug)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "debug", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (release)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (release, DatabaseAtomic)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (address)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (thread)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "thread", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (memory)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "memory", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (ubsan)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "undefined", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (debug)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "debug", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (release)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (unbundled)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "unbundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (release, polymorphic parts enabled)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (release, DatabaseAtomic)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Stress test (address)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Stress test (thread)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "thread", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Stress test (undefined)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "undefined", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Stress test (memory)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "memory", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + } +} From a88e391bd2f6812bd7d39db267f2b5bc77b0daa3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 20 Jul 2020 06:21:18 +0000 Subject: [PATCH 0015/1911] Fix bug with insert, simplify exchanges logic --- .../RabbitMQ/RabbitMQBlockOutputStream.cpp | 3 + .../ReadBufferFromRabbitMQConsumer.cpp | 246 +++++++----------- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 9 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 38 ++- src/Storages/RabbitMQ/StorageRabbitMQ.h | 5 +- .../WriteBufferToRabbitMQProducer.cpp | 52 ++-- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 15 +- .../integration/test_storage_rabbitmq/test.py | 130 +++++---- 8 files changed, 236 insertions(+), 262 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index 1a03fc4969e..2559b31c44a 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -56,6 +56,9 @@ void RabbitMQBlockOutputStream::write(const Block & block) void RabbitMQBlockOutputStream::writeSuffix() { child->writeSuffix(); + + if (buffer) + buffer->finilizeProducer(); } } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 6bd5c36e757..d59e9c9eade 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -14,20 +14,9 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - namespace ExchangeType { - /// Note that default here means default by implementation and not by rabbitmq settings - static const String DEFAULT = "default"; - static const String FANOUT = "fanout"; - static const String DIRECT = "direct"; - static const String TOPIC = "topic"; - static const String HASH = "consistent_hash"; - static const String HEADERS = "headers"; + static const String HASH_SUF = "_hash"; } static const auto QUEUE_SIZE = 50000; /// Equals capacity of a single rabbitmq queue @@ -36,34 +25,31 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ChannelPtr consumer_channel_, HandlerPtr event_handler_, const String & exchange_name_, + const AMQP::ExchangeType & exchange_type_, const Names & routing_keys_, size_t channel_id_, Poco::Logger * log_, char row_delimiter_, - bool bind_by_id_, + bool hash_exchange_, size_t num_queues_, - const String & exchange_type_, const String & local_exchange_, const std::atomic & stopped_) : ReadBuffer(nullptr, 0) , consumer_channel(std::move(consumer_channel_)) , event_handler(event_handler_) , exchange_name(exchange_name_) + , exchange_type(exchange_type_) , routing_keys(routing_keys_) , channel_id(channel_id_) - , bind_by_id(bind_by_id_) + , hash_exchange(hash_exchange_) , num_queues(num_queues_) - , exchange_type(exchange_type_) , local_exchange(local_exchange_) - , local_default_exchange(local_exchange + "_" + ExchangeType::DIRECT) - , local_hash_exchange(local_exchange + "_" + ExchangeType::HASH) + , local_hash_exchange(local_exchange + ExchangeType::HASH_SUF) , log(log_) , row_delimiter(row_delimiter_) , stopped(stopped_) , messages(QUEUE_SIZE * num_queues) { - exchange_type_set = exchange_type != ExchangeType::DEFAULT; - /* One queue per consumer can handle up to 50000 messages. More queues per consumer can be added. * By default there is one queue per consumer. */ @@ -86,67 +72,24 @@ ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() void ReadBufferFromRabbitMQConsumer::initExchange() { - /* This direct-exchange is used for default implemenation and for INSERT query (so it is always declared). If exchange_type - * is not set, then there are only two exchanges - external, defined by the client, and local, unique for each table (default). - * This strict division to external and local exchanges is needed to avoid too much complexity with defining exchange_name - * for INSERT query producer and, in general, it is better to distinguish them into separate ones. - */ - consumer_channel->declareExchange(local_default_exchange, AMQP::direct).onError([&](const char * message) - { - local_exchange_declared = false; - LOG_ERROR(log, "Failed to declare local direct-exchange. Reason: {}", message); - }); - - if (!exchange_type_set) - { - consumer_channel->declareExchange(exchange_name, AMQP::fanout).onError([&](const char * message) - { - local_exchange_declared = false; - LOG_ERROR(log, "Failed to declare default fanout-exchange. Reason: {}", message); - }); - - /// With fanout exchange the binding key is ignored - a parameter might be arbitrary. All distribution lies on local_exchange. - consumer_channel->bindExchange(exchange_name, local_default_exchange, routing_keys[0]).onError([&](const char * message) - { - local_exchange_declared = false; - LOG_ERROR(log, "Failed to bind local direct-exchange to fanout-exchange. Reason: {}", message); - }); - - return; - } - - AMQP::ExchangeType type; - if (exchange_type == ExchangeType::FANOUT) type = AMQP::ExchangeType::fanout; - else if (exchange_type == ExchangeType::DIRECT) type = AMQP::ExchangeType::direct; - else if (exchange_type == ExchangeType::TOPIC) type = AMQP::ExchangeType::topic; - else if (exchange_type == ExchangeType::HASH) type = AMQP::ExchangeType::consistent_hash; - else if (exchange_type == ExchangeType::HEADERS) type = AMQP::ExchangeType::headers; - else throw Exception("Invalid exchange type", ErrorCodes::BAD_ARGUMENTS); - /* Declare client's exchange of the specified type and bind it to hash-exchange (if it is not already hash-exchange), which - * will evenly distribute messages between all consumers. (This enables better scaling as without hash-exchange - the only - * option to avoid getting the same messages more than once - is having only one consumer with one queue) + * will evenly distribute messages between all consumers. */ - consumer_channel->declareExchange(exchange_name, type).onError([&](const char * message) + consumer_channel->declareExchange(exchange_name, exchange_type).onError([&](const char * message) { local_exchange_declared = false; LOG_ERROR(log, "Failed to declare client's {} exchange. Reason: {}", exchange_type, message); }); /// No need for declaring hash-exchange if there is only one consumer with one queue or exchange type is already hash - if (!bind_by_id) + if (!hash_exchange || exchange_type == AMQP::ExchangeType::consistent_hash) return; - hash_exchange = true; - - if (exchange_type == ExchangeType::HASH) - return; - - /* By default hash exchange distributes messages based on a hash value of a routing key, which must be a string integer. But - * in current case we use hash exchange for binding to another exchange of some other type, which needs its own routing keys - * of other types: headers, patterns and string-keys. This means that hash property must be changed. - */ { + /* By default hash exchange distributes messages based on a hash value of a routing key, which must be a string integer. But + * in current case we use hash exchange for binding to another exchange of some other type, which needs its own routing keys + * of other types: headers, patterns and string-keys. This means that hash property must be changed. + */ AMQP::Table binding_arguments; binding_arguments["hash-property"] = "message_id"; @@ -161,7 +104,7 @@ void ReadBufferFromRabbitMQConsumer::initExchange() /// Then bind client's exchange to sharding exchange (by keys, specified by the client): - if (exchange_type == ExchangeType::HEADERS) + if (exchange_type == AMQP::ExchangeType::headers) { AMQP::Table binding_arguments; std::vector matching; @@ -181,6 +124,14 @@ void ReadBufferFromRabbitMQConsumer::initExchange() LOG_ERROR(log, "Failed to bind local hash exchange to client's exchange. Reason: {}", message); }); } + else if (exchange_type == AMQP::ExchangeType::fanout) + { + consumer_channel->bindExchange(exchange_name, local_hash_exchange, routing_keys[0]).onError([&](const char * message) + { + local_exchange_declared = false; + LOG_ERROR(log, "Failed to bind local hash exchange to client's exchange. Reason: {}", message); + }); + } else { for (const auto & routing_key : routing_keys) @@ -198,30 +149,31 @@ void ReadBufferFromRabbitMQConsumer::initExchange() void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) { /// These variables might be updated later from a separate thread in onError callbacks. - if (!local_exchange_declared || (exchange_type_set && !local_hash_exchange_declared)) + if (!local_exchange_declared || (hash_exchange && !local_hash_exchange_declared)) { initExchange(); local_exchange_declared = true; local_hash_exchange_declared = true; } - bool default_bindings_created = false, default_bindings_error = false; bool bindings_created = false, bindings_error = false; consumer_channel->declareQueue(AMQP::exclusive) .onSuccess([&](const std::string & queue_name_, int /* msgcount */, int /* consumercount */) { queues.emplace_back(queue_name_); + LOG_DEBUG(log, "Queue " + queue_name_ + " is declared"); + subscribed_queue[queue_name_] = false; - - String binding_key = routing_keys[0]; - - /* Every consumer has at least one unique queue. Bind the queues to exchange based on the consumer_channel_id - * in case there is one queue per consumer and bind by queue_id in case there is more than 1 queue per consumer. - * (queue_id is based on channel_id) + /* Subscription can probably be moved back to readPrefix(), but not sure whether it is better in regard to speed, because + * if moved there, it must(!) be wrapped inside a channel->onSuccess callback or any other, otherwise + * consumer might fail to subscribe and no resubscription will help. */ - if (bind_by_id || hash_exchange) + subscribe(queues.back()); + + if (hash_exchange) { + String binding_key; if (queues.size() == 1) { binding_key = std::to_string(channel_id); @@ -230,39 +182,67 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) { binding_key = std::to_string(channel_id + queue_id); } + /* If exchange_type == hash, then bind directly to this client's exchange (because there is no need for a distributor + * exchange as it is already hash-exchange), otherwise hash-exchange is a local distributor exchange. + */ + String current_hash_exchange = exchange_type == AMQP::ExchangeType::consistent_hash ? exchange_name : local_hash_exchange; + + /// If hash-exchange is used for messages distribution, then the binding key is ignored - can be arbitrary. + consumer_channel->bindQueue(current_hash_exchange, queue_name_, binding_key) + .onSuccess([&] + { + bindings_created = true; + }) + .onError([&](const char * message) + { + bindings_error = true; + LOG_ERROR(log, "Failed to create queue binding. Reason: {}", message); + }); } - - /// Bind queue to exchange that is used for INSERT query and also for default implementation. - consumer_channel->bindQueue(local_default_exchange, queue_name_, binding_key) - .onSuccess([&] + else if (exchange_type == AMQP::ExchangeType::fanout) { - default_bindings_created = true; - }) - .onError([&](const char * message) - { - default_bindings_error = true; - LOG_ERROR(log, "Failed to bind to key {}. Reason: {}", binding_key, message); - }); - - /* Subscription can probably be moved back to readPrefix(), but not sure whether it is better in regard to speed, because - * if moved there, it must(!) be wrapped inside a channel->onSuccess callback or any other, otherwise - * consumer might fail to subscribe and no resubscription will help. - */ - subscribe(queues.back()); - - LOG_DEBUG(log, "Queue " + queue_name_ + " is declared"); - - if (exchange_type_set) - { - if (hash_exchange) + consumer_channel->bindQueue(exchange_name, queue_name_, routing_keys[0]) + .onSuccess([&] { - /* If exchange_type == hash, then bind directly to this client's exchange (because there is no need for a distributor - * exchange as it is already hash-exchange), otherwise hash-exchange is a local distributor exchange. - */ - String current_hash_exchange = exchange_type == ExchangeType::HASH ? exchange_name : local_hash_exchange; + bindings_created = true; + }) + .onError([&](const char * message) + { + bindings_error = true; + LOG_ERROR(log, "Failed to bind to key. Reason: {}", message); + }); + } + else if (exchange_type == AMQP::ExchangeType::headers) + { + AMQP::Table binding_arguments; + std::vector matching; - /// If hash-exchange is used for messages distribution, then the binding key is ignored - can be arbitrary. - consumer_channel->bindQueue(current_hash_exchange, queue_name_, binding_key) + /// It is not parsed for the second time - if it was parsed above, then it would never end up here. + for (const auto & header : routing_keys) + { + boost::split(matching, header, [](char c){ return c == '='; }); + binding_arguments[matching[0]] = matching[1]; + matching.clear(); + } + + consumer_channel->bindQueue(exchange_name, queue_name_, routing_keys[0], binding_arguments) + .onSuccess([&] + { + bindings_created = true; + }) + .onError([&](const char * message) + { + bindings_error = true; + LOG_ERROR(log, "Failed to bind queue. Reason: {}", message); + }); + } + else + { + /// Means there is only one queue with one consumer - no even distribution needed - no hash-exchange. + for (const auto & routing_key : routing_keys) + { + /// Binding directly to exchange, specified by the client. + consumer_channel->bindQueue(exchange_name, queue_name_, routing_key) .onSuccess([&] { bindings_created = true; @@ -270,56 +250,14 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) .onError([&](const char * message) { bindings_error = true; - LOG_ERROR(log, "Failed to create queue binding to key {}. Reason: {}", binding_key, message); + LOG_ERROR(log, "Failed to bind queue. Reason: {}", message); }); } - else if (exchange_type == ExchangeType::HEADERS) - { - AMQP::Table binding_arguments; - std::vector matching; - - /// It is not parsed for the second time - if it was parsed above, then it would never end up here. - for (const auto & header : routing_keys) - { - boost::split(matching, header, [](char c){ return c == '='; }); - binding_arguments[matching[0]] = matching[1]; - matching.clear(); - } - - consumer_channel->bindQueue(exchange_name, queue_name_, routing_keys[0], binding_arguments) - .onSuccess([&] - { - bindings_created = true; - }) - .onError([&](const char * message) - { - bindings_error = true; - LOG_ERROR(log, "Failed to bind queue to key. Reason: {}", message); - }); - } - else - { - /// Means there is only one queue with one consumer - no even distribution needed - no hash-exchange. - for (const auto & routing_key : routing_keys) - { - /// Binding directly to exchange, specified by the client. - consumer_channel->bindQueue(exchange_name, queue_name_, routing_key) - .onSuccess([&] - { - bindings_created = true; - }) - .onError([&](const char * message) - { - bindings_error = true; - LOG_ERROR(log, "Failed to bind queue to key. Reason: {}", message); - }); - } - } } }) .onError([&](const char * message) { - default_bindings_error = true; + bindings_error = true; LOG_ERROR(log, "Failed to declare queue on the channel. Reason: {}", message); }); @@ -327,7 +265,7 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) * It is important at this moment to make sure that queue bindings are created before any publishing can happen because * otherwise messages will be routed nowhere. */ - while ((!default_bindings_created && !default_bindings_error) || (exchange_type_set && !bindings_created && !bindings_error)) + while (!bindings_created && !bindings_error) { iterateEventLoop(); } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 51ef8ceba3e..82dc3f55248 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -26,13 +26,13 @@ public: ChannelPtr consumer_channel_, HandlerPtr event_handler_, const String & exchange_name_, + const AMQP::ExchangeType & exchange_type_, const Names & routing_keys_, size_t channel_id_, Poco::Logger * log_, char row_delimiter_, - bool bind_by_id_, + bool hash_exchange_, size_t num_queues_, - const String & exchange_type_, const String & local_exchange_, const std::atomic & stopped_); @@ -48,12 +48,12 @@ private: HandlerPtr event_handler; const String exchange_name; + const AMQP::ExchangeType exchange_type; const Names routing_keys; const size_t channel_id; - const bool bind_by_id; + const bool hash_exchange; const size_t num_queues; - const String exchange_type; const String local_exchange; const String local_default_exchange; const String local_hash_exchange; @@ -65,7 +65,6 @@ private: String default_local_exchange; bool local_exchange_declared = false, local_hash_exchange_declared = false; - bool exchange_type_set = false, hash_exchange = false; std::atomic consumer_error = false; std::atomic count_subscribed = 0, wait_subscribed; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index a3d16087e34..0d6cf95f39c 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -49,6 +49,16 @@ namespace ErrorCodes extern const int CANNOT_CONNECT_RABBITMQ; } +namespace ExchangeType +{ + /// Note that default here means default by implementation and not by rabbitmq settings + static const String DEFAULT = "default"; + static const String FANOUT = "fanout"; + static const String DIRECT = "direct"; + static const String TOPIC = "topic"; + static const String HASH = "consistent_hash"; + static const String HEADERS = "headers"; +} StorageRabbitMQ::StorageRabbitMQ( const StorageID & table_id_, @@ -72,7 +82,6 @@ StorageRabbitMQ::StorageRabbitMQ( , row_delimiter(row_delimiter_) , num_consumers(num_consumers_) , num_queues(num_queues_) - , exchange_type(exchange_type_) , use_transactional_channel(use_transactional_channel_) , log(&Poco::Logger::get("StorageRabbitMQ (" + table_id_.table_name + ")")) , parsed_address(parseAddress(global_context.getMacros()->expand(host_port_), 5672)) @@ -107,7 +116,22 @@ StorageRabbitMQ::StorageRabbitMQ( heartbeat_task = global_context.getSchedulePool().createTask("RabbitMQHeartbeatTask", [this]{ heartbeatFunc(); }); heartbeat_task->deactivate(); - bind_by_id = num_consumers > 1 || num_queues > 1; + hash_exchange = num_consumers > 1 || num_queues > 1; + + exchange_type_set = exchange_type_ != ExchangeType::DEFAULT; + if (exchange_type_set) + { + if (exchange_type_ == ExchangeType::FANOUT) exchange_type = AMQP::ExchangeType::fanout; + else if (exchange_type_ == ExchangeType::DIRECT) exchange_type = AMQP::ExchangeType::direct; + else if (exchange_type_ == ExchangeType::TOPIC) exchange_type = AMQP::ExchangeType::topic; + else if (exchange_type_ == ExchangeType::HASH) exchange_type = AMQP::ExchangeType::consistent_hash; + else if (exchange_type_ == ExchangeType::HEADERS) exchange_type = AMQP::ExchangeType::headers; + else throw Exception("Invalid exchange type", ErrorCodes::BAD_ARGUMENTS); + } + else + { + exchange_type = AMQP::ExchangeType::fanout; + } auto table_id = getStorageID(); String table_name = table_id.table_name; @@ -264,17 +288,17 @@ ConsumerBufferPtr StorageRabbitMQ::createReadBuffer() ChannelPtr consumer_channel = std::make_shared(connection.get()); return std::make_shared( - consumer_channel, event_handler, exchange_name, routing_keys, - next_channel_id, log, row_delimiter, bind_by_id, num_queues, - exchange_type, local_exchange_name, stream_cancelled); + consumer_channel, event_handler, exchange_name, exchange_type, routing_keys, + next_channel_id, log, row_delimiter, hash_exchange, num_queues, + local_exchange_name, stream_cancelled); } ProducerBufferPtr StorageRabbitMQ::createWriteBuffer() { return std::make_shared( - parsed_address, global_context, login_password, routing_keys[0], local_exchange_name, - log, num_consumers * num_queues, bind_by_id, use_transactional_channel, + parsed_address, global_context, login_password, routing_keys, exchange_name, exchange_type, + log, num_consumers * num_queues, use_transactional_channel, row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index e1c8b33c91e..4457c5ff8c9 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -76,15 +76,15 @@ private: Names routing_keys; const String exchange_name; + AMQP::ExchangeType exchange_type; String local_exchange_name; const String format_name; char row_delimiter; size_t num_consumers; size_t num_created_consumers = 0; - bool bind_by_id; + bool hash_exchange; size_t num_queues; - const String exchange_type; const bool use_transactional_channel; Poco::Logger * log; @@ -99,6 +99,7 @@ private: std::mutex mutex; std::vector buffers; /// available buffers for RabbitMQ consumers + bool exchange_type_set = false; size_t next_channel_id = 1; /// Must >= 1 because it is used as a binding key, which has to be > 0 bool update_channel_id = false; std::atomic loop_started = false; diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 57ef2405255..11b13714448 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -16,6 +16,7 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_CONNECT_RABBITMQ; + extern const int LOGICAL_ERROR; } static const auto QUEUE_SIZE = 50000; @@ -27,20 +28,20 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( std::pair & parsed_address, Context & global_context, const std::pair & login_password_, - const String & routing_key_, - const String & exchange_, + const Names & routing_keys_, + const String & exchange_name_, + const AMQP::ExchangeType exchange_type_, Poco::Logger * log_, size_t num_queues_, - bool bind_by_id_, bool use_transactional_channel_, std::optional delimiter, size_t rows_per_message, size_t chunk_size_) : WriteBuffer(nullptr, 0) , login_password(login_password_) - , routing_key(routing_key_) - , exchange_name(exchange_ + "_direct") - , bind_by_id(bind_by_id_) + , routing_keys(routing_keys_) + , exchange_name(exchange_name_) + , exchange_type(exchange_type_) , num_queues(num_queues_) , use_transactional_channel(use_transactional_channel_) , payloads(QUEUE_SIZE * num_queues) @@ -73,7 +74,6 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( } producer_channel = std::make_shared(connection.get()); - checkExchange(); /// If publishing should be wrapped in transactions if (use_transactional_channel) @@ -83,6 +83,17 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); writing_task->deactivate(); + + if (exchange_type == AMQP::ExchangeType::headers) + { + std::vector matching; + for (const auto & header : routing_keys) + { + boost::split(matching, header, [](char c){ return c == '='; }); + key_arguments[matching[0]] = matching[1]; + matching.clear(); + } + } } @@ -90,7 +101,7 @@ WriteBufferToRabbitMQProducer::~WriteBufferToRabbitMQProducer() { stop_loop.store(true); writing_task->deactivate(); - checkExchange(); + initExchange(); connection->close(); assert(rows == 0 && chunks.empty()); @@ -133,28 +144,34 @@ void WriteBufferToRabbitMQProducer::writingFunc() while (!payloads.empty()) { payloads.pop(payload); - next_queue = next_queue % num_queues + 1; - if (bind_by_id) + if (exchange_type == AMQP::ExchangeType::consistent_hash) { + next_queue = next_queue % num_queues + 1; producer_channel->publish(exchange_name, std::to_string(next_queue), payload); } + else if (exchange_type == AMQP::ExchangeType::headers) + { + AMQP::Envelope envelope(payload.data(), payload.size()); + envelope.setHeaders(key_arguments); + producer_channel->publish(exchange_name, "", envelope, key_arguments); + } else { - producer_channel->publish(exchange_name, routing_key, payload); + producer_channel->publish(exchange_name, routing_keys[0], payload); } } + iterateEventLoop(); } } -void WriteBufferToRabbitMQProducer::checkExchange() +void WriteBufferToRabbitMQProducer::initExchange() { std::atomic exchange_declared = false, exchange_error = false; - /// The AMQP::passive flag indicates that it should only be checked if there is a valid exchange with the given name. - producer_channel->declareExchange(exchange_name, AMQP::direct, AMQP::passive) + producer_channel->declareExchange(exchange_name, exchange_type) .onSuccess([&]() { exchange_declared = true; @@ -162,10 +179,10 @@ void WriteBufferToRabbitMQProducer::checkExchange() .onError([&](const char * message) { exchange_error = true; - LOG_ERROR(log, "Exchange for INSERT query was not declared. Reason: {}", message); + LOG_ERROR(log, "Exchange error: {}", message); }); - /// These variables are updated in a separate thread and starting the loop blocks current thread + /// These variables are updated in a separate thread. while (!exchange_declared && !exchange_error) { iterateEventLoop(); @@ -175,9 +192,6 @@ void WriteBufferToRabbitMQProducer::checkExchange() void WriteBufferToRabbitMQProducer::finilizeProducer() { - /// This will make sure everything is published - checkExchange(); - if (use_transactional_channel) { std::atomic answer_received = false, wait_rollback = false; diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 8dc5a32b7d7..20b133b6930 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -23,11 +23,11 @@ public: std::pair & parsed_address, Context & global_context, const std::pair & login_password_, - const String & routing_key_, - const String & exchange_, + const Names & routing_keys_, + const String & exchange_name_, + const AMQP::ExchangeType exchange_type_, Poco::Logger * log_, size_t num_queues_, - bool bind_by_id_, bool use_transactional_channel_, std::optional delimiter, size_t rows_per_message, @@ -38,21 +38,22 @@ public: void countRow(); void activateWriting() { writing_task->activateAndSchedule(); } + void finilizeProducer(); private: void nextImpl() override; - void checkExchange(); + void initExchange(); void iterateEventLoop(); void writingFunc(); - void finilizeProducer(); const std::pair login_password; - const String routing_key; + const Names routing_keys; const String exchange_name; - const bool bind_by_id; + AMQP::ExchangeType exchange_type; const size_t num_queues; const bool use_transactional_channel; + AMQP::Table key_arguments; BackgroundSchedulePool::TaskHolder writing_task; std::atomic stop_loop = false; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 42b7101f9c6..a044eba805c 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -485,7 +485,6 @@ def test_rabbitmq_big_message(rabbitmq_cluster): while True: result = instance.query('SELECT count() FROM test.view') - print("Result", result, "Expected", batch_messages * rabbitmq_messages) if int(result) == batch_messages * rabbitmq_messages: break @@ -552,7 +551,6 @@ def test_rabbitmq_sharding_between_channels_publish(rabbitmq_cluster): while True: result = instance.query('SELECT count() FROM test.view') time.sleep(1) - print("Result", result, "Expected", messages_num * threads_num) if int(result) == messages_num * threads_num: break @@ -778,6 +776,7 @@ def test_rabbitmq_insert(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'insert', + rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'insert1', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; @@ -788,10 +787,64 @@ def test_rabbitmq_insert(rabbitmq_cluster): consumer_connection = pika.BlockingConnection(parameters) consumer = consumer_connection.channel() - consumer.exchange_declare(exchange='insert_rabbitmq_direct', exchange_type='direct') + consumer.exchange_declare(exchange='insert', exchange_type='direct') result = consumer.queue_declare(queue='') queue_name = result.method.queue - consumer.queue_bind(exchange='insert_rabbitmq_direct', queue=queue_name, routing_key='insert1') + consumer.queue_bind(exchange='insert', queue=queue_name, routing_key='insert1') + + values = [] + for i in range(50): + values.append("({i}, {i})".format(i=i)) + values = ','.join(values) + + while True: + try: + instance.query("INSERT INTO test.rabbitmq VALUES {}".format(values)) + break + except QueryRuntimeException as e: + if 'Local: Timed out.' in str(e): + continue + else: + raise + + insert_messages = [] + def onReceived(channel, method, properties, body): + i = 0 + insert_messages.append(body.decode()) + if (len(insert_messages) == 50): + channel.stop_consuming() + + consumer.basic_consume(onReceived, queue_name) + consumer.start_consuming() + consumer_connection.close() + + result = '\n'.join(insert_messages) + rabbitmq_check_result(result, True) + + +@pytest.mark.timeout(240) +def test_rabbitmq_insert_headers_exchange(rabbitmq_cluster): + instance.query(''' + CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'insert_headers', + rabbitmq_exchange_type = 'headers', + rabbitmq_routing_key_list = 'test=insert,topic=headers', + rabbitmq_format = 'TSV', + rabbitmq_row_delimiter = '\\n'; + ''') + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + consumer_connection = pika.BlockingConnection(parameters) + + consumer = consumer_connection.channel() + consumer.exchange_declare(exchange='insert_headers', exchange_type='headers') + result = consumer.queue_declare(queue='') + queue_name = result.method.queue + consumer.queue_bind(exchange='insert_headers', queue=queue_name, routing_key="", + arguments={'x-match':'all', 'test':'insert', 'topic':'headers'}) values = [] for i in range(50): @@ -815,7 +868,6 @@ def test_rabbitmq_insert(rabbitmq_cluster): if (len(insert_messages) == 50): channel.stop_consuming() - consumer.basic_qos(prefetch_count=50) consumer.basic_consume(onReceived, queue_name) consumer.start_consuming() consumer_connection.close() @@ -833,6 +885,8 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): CREATE TABLE test.rabbitmq_many (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'many_inserts', + rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'insert2', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; @@ -887,69 +941,6 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): assert int(result) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) -@pytest.mark.timeout(240) -def test_rabbitmq_sharding_between_channels_and_queues_insert(rabbitmq_cluster): - instance.query(''' - DROP TABLE IF EXISTS test.view_sharding; - DROP TABLE IF EXISTS test.consumer_sharding; - CREATE TABLE test.rabbitmq_sharding (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_num_consumers = 5, - rabbitmq_num_queues = 2, - rabbitmq_format = 'TSV', - rabbitmq_row_delimiter = '\\n'; - CREATE TABLE test.view_sharding (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key - SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3; - CREATE MATERIALIZED VIEW test.consumer_sharding TO test.view_sharding AS - SELECT * FROM test.rabbitmq_sharding; - ''') - - messages_num = 10000 - def insert(): - values = [] - for i in range(messages_num): - values.append("({i}, {i})".format(i=i)) - values = ','.join(values) - - while True: - try: - instance.query("INSERT INTO test.rabbitmq_sharding VALUES {}".format(values)) - break - except QueryRuntimeException as e: - if 'Local: Timed out.' in str(e): - continue - else: - raise - - threads = [] - threads_num = 20 - for _ in range(threads_num): - threads.append(threading.Thread(target=insert)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - while True: - result = instance.query('SELECT count() FROM test.view_sharding') - time.sleep(1) - if int(result) == messages_num * threads_num: - break - - instance.query(''' - DROP TABLE IF EXISTS test.rabbitmq_sharding; - DROP TABLE IF EXISTS test.consumer_sharding; - DROP TABLE IF EXISTS test.view_sharding; - ''') - - for thread in threads: - thread.join() - - assert int(result) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) - - @pytest.mark.timeout(420) def test_rabbitmq_overloaded_insert(rabbitmq_cluster): instance.query(''' @@ -958,6 +949,9 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): CREATE TABLE test.rabbitmq_overload (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'over', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'over', rabbitmq_num_consumers = 10, rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; From ac448db4918ad03cafed1ec2616af0bbb759fbcd Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 20 Jul 2020 10:05:00 +0000 Subject: [PATCH 0016/1911] Add virtuals --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 9 +- .../ReadBufferFromRabbitMQConsumer.cpp | 17 +-- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 15 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 5 +- .../integration/test_storage_rabbitmq/test.py | 134 ++++++++++++++++++ 5 files changed, 168 insertions(+), 12 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index 83e3a02b478..7b1cdd11317 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -21,7 +21,8 @@ RabbitMQBlockInputStream::RabbitMQBlockInputStream( , context(context_) , column_names(columns) , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) - , virtual_header(metadata_snapshot->getSampleBlockForColumns({"_exchange"}, storage.getVirtuals(), storage.getStorageID())) + , virtual_header(metadata_snapshot->getSampleBlockForColumns( + {"_exchange_name", "_consumer_tag", "_delivery_tag", "_redelivered"}, storage.getVirtuals(), storage.getStorageID())) { } @@ -124,10 +125,16 @@ Block RabbitMQBlockInputStream::readImpl() auto new_rows = read_rabbitmq_message(); auto exchange_name = buffer->getExchange(); + auto consumer_tag = buffer->getConsumerTag(); + auto delivery_tag = buffer->getDeliveryTag(); + auto redelivered = buffer->getRedelivered(); for (size_t i = 0; i < new_rows; ++i) { virtual_columns[0]->insert(exchange_name); + virtual_columns[1]->insert(consumer_tag); + virtual_columns[2]->insert(delivery_tag); + virtual_columns[3]->insert(redelivered); } total_rows = total_rows + new_rows; diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index d59e9c9eade..0aff21f8a8e 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -48,7 +48,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , log(log_) , row_delimiter(row_delimiter_) , stopped(stopped_) - , messages(QUEUE_SIZE * num_queues) + , received(QUEUE_SIZE * num_queues) { /* One queue per consumer can handle up to 50000 messages. More queues per consumer can be added. * By default there is one queue per consumer. @@ -65,7 +65,7 @@ ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() { consumer_channel->close(); - messages.clear(); + received.clear(); BufferBase::set(nullptr, 0, 0); } @@ -278,15 +278,16 @@ void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name) return; consumer_channel->consume(queue_name, AMQP::noack) - .onSuccess([&](const std::string & /* consumer */) + .onSuccess([&](const std::string & consumer) { subscribed_queue[queue_name] = true; consumer_error = false; ++count_subscribed; + consumer_tag = consumer; LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); }) - .onReceived([&](const AMQP::Message & message, uint64_t /* deliveryTag */, bool /* redelivered */) + .onReceived([&](const AMQP::Message & message, uint64_t deliveryTag, bool redelivered) { size_t message_size = message.bodySize(); if (message_size && message.body() != nullptr) @@ -297,7 +298,7 @@ void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name) message_received += row_delimiter; } - messages.push(message_received); + received.push({deliveryTag, message_received, redelivered}); } }) .onError([&](const char * message) @@ -346,10 +347,10 @@ bool ReadBufferFromRabbitMQConsumer::nextImpl() if (stopped || !allowed) return false; - if (messages.tryPop(current)) + if (received.tryPop(current)) { - auto * new_position = const_cast(current.data()); - BufferBase::set(new_position, current.size(), 0); + auto * new_position = const_cast(current.message.data()); + BufferBase::set(new_position, current.message.size(), 0); allowed = false; return true; diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 82dc3f55248..85644562d0c 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -38,10 +38,20 @@ public: ~ReadBufferFromRabbitMQConsumer() override; + struct MessageData + { + UInt64 delivery_tag; + String message; + bool redelivered; + }; + void allowNext() { allowed = true; } // Allow to read next message. void checkSubscription(); auto getExchange() const { return exchange_name; } + auto getConsumerTag() const { return consumer_tag; } + auto getDeliveryTag() const { return current.delivery_tag; } + auto getRedelivered() const { return current.redelivered; } private: ChannelPtr consumer_channel; @@ -69,8 +79,9 @@ private: std::atomic consumer_error = false; std::atomic count_subscribed = 0, wait_subscribed; - ConcurrentBoundedQueue messages; - String current; + String consumer_tag; + ConcurrentBoundedQueue received; + MessageData current; std::vector queues; std::unordered_map subscribed_queue; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 0d6cf95f39c..66af7dc3f56 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -597,7 +597,10 @@ void registerStorageRabbitMQ(StorageFactory & factory) NamesAndTypesList StorageRabbitMQ::getVirtuals() const { return NamesAndTypesList{ - {"_exchange", std::make_shared()} + {"_exchange_name", std::make_shared()}, + {"_consumer_tag", std::make_shared()}, + {"_delivery_tag", std::make_shared()}, + {"_redelivered", std::make_shared()} }; } diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index a044eba805c..eaaa8613b5f 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -1520,6 +1520,140 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): assert int(result) == messages_num * num_tables_to_receive, 'ClickHouse lost some messages: {}'.format(result) +@pytest.mark.timeout(420) +def test_rabbitmq_virtual_columns(rabbitmq_cluster): + instance.query(''' + DROP TABLE IF EXISTS test.view; + CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'virtuals', + rabbitmq_format = 'JSONEachRow'; + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT value, key, _exchange_name, _consumer_tag, _delivery_tag, _redelivered FROM test.rabbitmq; + ''') + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + channel.exchange_declare(exchange='virtuals', exchange_type='fanout') + + message_num = 10 + i = [0] + messages = [] + for _ in range(message_num): + messages.append(json.dumps({'key': i[0], 'value': i[0]})) + i[0] += 1 + + for message in messages: + channel.basic_publish(exchange='virtuals', routing_key='', body=message) + + while True: + result = instance.query('SELECT count() FROM test.view') + time.sleep(1) + if int(result) == message_num: + break + + connection.close() + + result = instance.query("SELECT count(DISTINCT _delivery_tag) FROM test.view") + assert int(result) == 10 + + result = instance.query("SELECT count(DISTINCT _consumer_tag) FROM test.view") + assert int(result) == 1 + + result = instance.query(''' + SELECT key, value, _exchange_name, SUBSTRING(_consumer_tag, 1, 8), _delivery_tag, _redelivered + FROM test.view + ORDER BY key + ''') + + expected = '''\ +0 0 virtuals amq.ctag 1 0 +1 1 virtuals amq.ctag 2 0 +2 2 virtuals amq.ctag 3 0 +3 3 virtuals amq.ctag 4 0 +4 4 virtuals amq.ctag 5 0 +5 5 virtuals amq.ctag 6 0 +6 6 virtuals amq.ctag 7 0 +7 7 virtuals amq.ctag 8 0 +8 8 virtuals amq.ctag 9 0 +9 9 virtuals amq.ctag 10 0 +''' + assert TSV(result) == TSV(expected) + + +@pytest.mark.timeout(420) +def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): + instance.query(''' + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'virtuals_mv', + rabbitmq_format = 'JSONEachRow'; + CREATE TABLE test.view (key UInt64, value UInt64, + exchange_name String, consumer_tag String, delivery_tag UInt64, redelivered UInt8) ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT *, _exchange_name as exchange_name, _consumer_tag as consumer_tag, _delivery_tag as delivery_tag, _redelivered as redelivered + FROM test.rabbitmq; + ''') + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + channel.exchange_declare(exchange='virtuals_mv', exchange_type='fanout') + + message_num = 10 + i = [0] + messages = [] + for _ in range(message_num): + messages.append(json.dumps({'key': i[0], 'value': i[0]})) + i[0] += 1 + + for message in messages: + channel.basic_publish(exchange='virtuals_mv', routing_key='', body=message) + + while True: + result = instance.query('SELECT count() FROM test.view') + time.sleep(1) + if int(result) == message_num: + break + + connection.close() + + result = instance.query("SELECT count(DISTINCT delivery_tag) FROM test.view") + assert int(result) == 10 + + result = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") + assert int(result) == 1 + + result = instance.query("SELECT key, value, exchange_name, SUBSTRING(consumer_tag, 1, 8), delivery_tag, redelivered FROM test.view") + expected = '''\ +0 0 virtuals_mv amq.ctag 1 0 +1 1 virtuals_mv amq.ctag 2 0 +2 2 virtuals_mv amq.ctag 3 0 +3 3 virtuals_mv amq.ctag 4 0 +4 4 virtuals_mv amq.ctag 5 0 +5 5 virtuals_mv amq.ctag 6 0 +6 6 virtuals_mv amq.ctag 7 0 +7 7 virtuals_mv amq.ctag 8 0 +8 8 virtuals_mv amq.ctag 9 0 +9 9 virtuals_mv amq.ctag 10 0 +''' + + instance.query(''' + DROP TABLE test.consumer; + DROP TABLE test.view; + ''') + + assert TSV(result) == TSV(expected) + + if __name__ == '__main__': cluster.start() raw_input("Cluster created, press any key to destroy...") From f0f6111655852d1c5cc8c50db7e0efcdaed41192 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 Jul 2020 15:47:39 +0000 Subject: [PATCH 0017/1911] Move exchange init, add bridge-exchange --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 2 +- .../RabbitMQ/RabbitMQBlockOutputStream.cpp | 2 + src/Storages/RabbitMQ/RabbitMQHandler.cpp | 2 + src/Storages/RabbitMQ/RabbitMQHandler.h | 3 +- .../ReadBufferFromRabbitMQConsumer.cpp | 125 ++------------ .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 13 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 152 +++++++++++++++++- src/Storages/RabbitMQ/StorageRabbitMQ.h | 13 +- .../WriteBufferToRabbitMQProducer.cpp | 5 +- .../integration/test_storage_rabbitmq/test.py | 129 ++++++++------- 10 files changed, 253 insertions(+), 193 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index 7b1cdd11317..6e8e153392c 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -124,7 +124,7 @@ Block RabbitMQBlockInputStream::readImpl() auto new_rows = read_rabbitmq_message(); - auto exchange_name = buffer->getExchange(); + auto exchange_name = storage.getExchange(); auto consumer_tag = buffer->getConsumerTag(); auto delivery_tag = buffer->getDeliveryTag(); auto redelivered = buffer->getRedelivered(); diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index 2559b31c44a..87a17d3e1ed 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -33,6 +33,8 @@ Block RabbitMQBlockOutputStream::getHeader() const void RabbitMQBlockOutputStream::writePrefix() { + if (storage.checkBridge()) + storage.unbindExchange(); buffer = storage.createWriteBuffer(); if (!buffer) throw Exception("Failed to create RabbitMQ producer!", ErrorCodes::CANNOT_CREATE_IO_BUFFER); diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.cpp b/src/Storages/RabbitMQ/RabbitMQHandler.cpp index 5d17ff23b64..f01b1e60eab 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.cpp +++ b/src/Storages/RabbitMQ/RabbitMQHandler.cpp @@ -31,9 +31,11 @@ void RabbitMQHandler::onError(AMQP::TcpConnection * connection, const char * mes void RabbitMQHandler::startLoop() { std::lock_guard lock(startup_mutex); + loop_started.store(true); /// stop_loop variable is updated in a separate thread while (!stop_loop.load()) uv_run(loop, UV_RUN_NOWAIT); + loop_started.store(false); } void RabbitMQHandler::iterateLoop() diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.h b/src/Storages/RabbitMQ/RabbitMQHandler.h index 5893ace1d2f..b1b84e1d07a 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.h +++ b/src/Storages/RabbitMQ/RabbitMQHandler.h @@ -21,12 +21,13 @@ public: void stop() { stop_loop.store(true); } void startLoop(); void iterateLoop(); + bool checkLoop() const { return loop_started.load(); } private: uv_loop_t * loop; Poco::Logger * log; - std::atomic stop_loop = false; + std::atomic stop_loop = false, loop_started = false; std::mutex startup_mutex; }; diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 0aff21f8a8e..8c272e04691 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -14,15 +14,11 @@ namespace DB { -namespace ExchangeType -{ - static const String HASH_SUF = "_hash"; -} - static const auto QUEUE_SIZE = 50000; /// Equals capacity of a single rabbitmq queue ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ChannelPtr consumer_channel_, + ChannelPtr setup_channel_, HandlerPtr event_handler_, const String & exchange_name_, const AMQP::ExchangeType & exchange_type_, @@ -36,6 +32,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( const std::atomic & stopped_) : ReadBuffer(nullptr, 0) , consumer_channel(std::move(consumer_channel_)) + , setup_channel(setup_channel_) , event_handler(event_handler_) , exchange_name(exchange_name_) , exchange_type(exchange_type_) @@ -43,21 +40,14 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , channel_id(channel_id_) , hash_exchange(hash_exchange_) , num_queues(num_queues_) - , local_exchange(local_exchange_) - , local_hash_exchange(local_exchange + ExchangeType::HASH_SUF) , log(log_) , row_delimiter(row_delimiter_) , stopped(stopped_) + , local_exchange(local_exchange_) , received(QUEUE_SIZE * num_queues) { - /* One queue per consumer can handle up to 50000 messages. More queues per consumer can be added. - * By default there is one queue per consumer. - */ for (size_t queue_id = 0; queue_id < num_queues; ++queue_id) - { - /// Queue bingings must be declared before any publishing => it must be done here and not in readPrefix() initQueueBindings(queue_id); - } } @@ -70,125 +60,34 @@ ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() } -void ReadBufferFromRabbitMQConsumer::initExchange() -{ - /* Declare client's exchange of the specified type and bind it to hash-exchange (if it is not already hash-exchange), which - * will evenly distribute messages between all consumers. - */ - consumer_channel->declareExchange(exchange_name, exchange_type).onError([&](const char * message) - { - local_exchange_declared = false; - LOG_ERROR(log, "Failed to declare client's {} exchange. Reason: {}", exchange_type, message); - }); - - /// No need for declaring hash-exchange if there is only one consumer with one queue or exchange type is already hash - if (!hash_exchange || exchange_type == AMQP::ExchangeType::consistent_hash) - return; - - { - /* By default hash exchange distributes messages based on a hash value of a routing key, which must be a string integer. But - * in current case we use hash exchange for binding to another exchange of some other type, which needs its own routing keys - * of other types: headers, patterns and string-keys. This means that hash property must be changed. - */ - AMQP::Table binding_arguments; - binding_arguments["hash-property"] = "message_id"; - - /// Declare exchange for sharding. - consumer_channel->declareExchange(local_hash_exchange, AMQP::consistent_hash, binding_arguments) - .onError([&](const char * message) - { - local_exchange_declared = false; - LOG_ERROR(log, "Failed to declare {} exchange: {}", exchange_type, message); - }); - } - - /// Then bind client's exchange to sharding exchange (by keys, specified by the client): - - if (exchange_type == AMQP::ExchangeType::headers) - { - AMQP::Table binding_arguments; - std::vector matching; - - for (const auto & header : routing_keys) - { - boost::split(matching, header, [](char c){ return c == '='; }); - binding_arguments[matching[0]] = matching[1]; - matching.clear(); - } - - /// Routing key can be arbitrary here. - consumer_channel->bindExchange(exchange_name, local_hash_exchange, routing_keys[0], binding_arguments) - .onError([&](const char * message) - { - local_exchange_declared = false; - LOG_ERROR(log, "Failed to bind local hash exchange to client's exchange. Reason: {}", message); - }); - } - else if (exchange_type == AMQP::ExchangeType::fanout) - { - consumer_channel->bindExchange(exchange_name, local_hash_exchange, routing_keys[0]).onError([&](const char * message) - { - local_exchange_declared = false; - LOG_ERROR(log, "Failed to bind local hash exchange to client's exchange. Reason: {}", message); - }); - } - else - { - for (const auto & routing_key : routing_keys) - { - consumer_channel->bindExchange(exchange_name, local_hash_exchange, routing_key).onError([&](const char * message) - { - local_exchange_declared = false; - LOG_ERROR(log, "Failed to bind local hash exchange to client's exchange. Reason: {}", message); - }); - } - } -} - - void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) { - /// These variables might be updated later from a separate thread in onError callbacks. - if (!local_exchange_declared || (hash_exchange && !local_hash_exchange_declared)) - { - initExchange(); - local_exchange_declared = true; - local_hash_exchange_declared = true; - } - bool bindings_created = false, bindings_error = false; - consumer_channel->declareQueue(AMQP::exclusive) + setup_channel->declareQueue(AMQP::exclusive) .onSuccess([&](const std::string & queue_name_, int /* msgcount */, int /* consumercount */) { queues.emplace_back(queue_name_); LOG_DEBUG(log, "Queue " + queue_name_ + " is declared"); subscribed_queue[queue_name_] = false; - /* Subscription can probably be moved back to readPrefix(), but not sure whether it is better in regard to speed, because - * if moved there, it must(!) be wrapped inside a channel->onSuccess callback or any other, otherwise - * consumer might fail to subscribe and no resubscription will help. - */ subscribe(queues.back()); if (hash_exchange) { String binding_key; if (queues.size() == 1) - { binding_key = std::to_string(channel_id); - } else - { binding_key = std::to_string(channel_id + queue_id); - } + /* If exchange_type == hash, then bind directly to this client's exchange (because there is no need for a distributor * exchange as it is already hash-exchange), otherwise hash-exchange is a local distributor exchange. */ - String current_hash_exchange = exchange_type == AMQP::ExchangeType::consistent_hash ? exchange_name : local_hash_exchange; + String current_hash_exchange = exchange_type == AMQP::ExchangeType::consistent_hash ? exchange_name : local_exchange; /// If hash-exchange is used for messages distribution, then the binding key is ignored - can be arbitrary. - consumer_channel->bindQueue(current_hash_exchange, queue_name_, binding_key) + setup_channel->bindQueue(current_hash_exchange, queue_name_, binding_key) .onSuccess([&] { bindings_created = true; @@ -201,7 +100,7 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) } else if (exchange_type == AMQP::ExchangeType::fanout) { - consumer_channel->bindQueue(exchange_name, queue_name_, routing_keys[0]) + setup_channel->bindQueue(exchange_name, queue_name_, routing_keys[0]) .onSuccess([&] { bindings_created = true; @@ -225,7 +124,7 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) matching.clear(); } - consumer_channel->bindQueue(exchange_name, queue_name_, routing_keys[0], binding_arguments) + setup_channel->bindQueue(exchange_name, queue_name_, routing_keys[0], binding_arguments) .onSuccess([&] { bindings_created = true; @@ -242,7 +141,7 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) for (const auto & routing_key : routing_keys) { /// Binding directly to exchange, specified by the client. - consumer_channel->bindQueue(exchange_name, queue_name_, routing_key) + setup_channel->bindQueue(exchange_name, queue_name_, routing_key) .onSuccess([&] { bindings_created = true; @@ -261,10 +160,6 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) LOG_ERROR(log, "Failed to declare queue on the channel. Reason: {}", message); }); - /* Run event loop (which updates local variables in a separate thread) until bindings are created or failed to be created. - * It is important at this moment to make sure that queue bindings are created before any publishing can happen because - * otherwise messages will be routed nowhere. - */ while (!bindings_created && !bindings_error) { iterateEventLoop(); diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 85644562d0c..6896dd7f4b0 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -24,6 +24,7 @@ class ReadBufferFromRabbitMQConsumer : public ReadBuffer public: ReadBufferFromRabbitMQConsumer( ChannelPtr consumer_channel_, + ChannelPtr setup_channel_, HandlerPtr event_handler_, const String & exchange_name_, const AMQP::ExchangeType & exchange_type_, @@ -48,13 +49,13 @@ public: void allowNext() { allowed = true; } // Allow to read next message. void checkSubscription(); - auto getExchange() const { return exchange_name; } auto getConsumerTag() const { return consumer_tag; } auto getDeliveryTag() const { return current.delivery_tag; } auto getRedelivered() const { return current.redelivered; } private: ChannelPtr consumer_channel; + ChannelPtr setup_channel; HandlerPtr event_handler; const String exchange_name; @@ -64,18 +65,12 @@ private: const bool hash_exchange; const size_t num_queues; - const String local_exchange; - const String local_default_exchange; - const String local_hash_exchange; - Poco::Logger * log; char row_delimiter; bool allowed = true; const std::atomic & stopped; - String default_local_exchange; - bool local_exchange_declared = false, local_hash_exchange_declared = false; - + const String local_exchange; std::atomic consumer_error = false; std::atomic count_subscribed = 0, wait_subscribed; @@ -87,7 +82,7 @@ private: bool nextImpl() override; - void initExchange(); + void connectAlternateExchange(); void initQueueBindings(const size_t queue_id); void subscribe(const String & queue_name); void iterateEventLoop(); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 66af7dc3f56..d56a46c4f55 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -118,8 +118,7 @@ StorageRabbitMQ::StorageRabbitMQ( hash_exchange = num_consumers > 1 || num_queues > 1; - exchange_type_set = exchange_type_ != ExchangeType::DEFAULT; - if (exchange_type_set) + if (exchange_type_ != ExchangeType::DEFAULT) { if (exchange_type_ == ExchangeType::FANOUT) exchange_type = AMQP::ExchangeType::fanout; else if (exchange_type_ == ExchangeType::DIRECT) exchange_type = AMQP::ExchangeType::direct; @@ -133,11 +132,23 @@ StorageRabbitMQ::StorageRabbitMQ( exchange_type = AMQP::ExchangeType::fanout; } + if (exchange_type == AMQP::ExchangeType::headers) + { + std::vector matching; + for (const auto & header : routing_keys) + { + boost::split(matching, header, [](char c){ return c == '='; }); + bind_headers[matching[0]] = matching[1]; + matching.clear(); + } + } + auto table_id = getStorageID(); String table_name = table_id.table_name; /// Make sure that local exchange name is unique for each table and is not the same as client's exchange name - local_exchange_name = exchange_name + "_" + table_name; + local_exchange = exchange_name + "_" + table_name; + bridge_exchange = local_exchange + "_bridge"; /// One looping task for all consumers as they share the same connection == the same handler == the same event loop looping_task = global_context.getSchedulePool().createTask("RabbitMQLoopingTask", [this]{ loopingFunc(); }); @@ -163,6 +174,133 @@ void StorageRabbitMQ::loopingFunc() } +void StorageRabbitMQ::initExchange() +{ + /* Declare client's exchange of the specified type and bind it to hash-exchange (if it is not already hash-exchange), which + * will evenly distribute messages between all consumers. + */ + setup_channel->declareExchange(exchange_name, exchange_type, AMQP::durable) + .onError([&](const char * message) + { + throw Exception("Unable to declare exchange. Make sure specified exchange is not already declared. Error: " + + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + }); + + /// Bridge exchange is needed to easily disconnect consumer queues. + setup_channel->declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable + AMQP::autodelete) + .onError([&](const char * message) + { + throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + }); + + if (!hash_exchange) + { + consumer_exchange = bridge_exchange; + return; + } + + /// Declare exchange for sharding. + AMQP::Table binding_arguments; + binding_arguments["hash-property"] = "message_id"; + + setup_channel->declareExchange(local_exchange, AMQP::consistent_hash, AMQP::durable + AMQP::autodelete, binding_arguments) + .onError([&](const char * message) + { + throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + }); + + setup_channel->bindExchange(bridge_exchange, local_exchange, routing_keys[0]) + .onError([&](const char * message) + { + throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + }); + + consumer_exchange = local_exchange; +} + + +void StorageRabbitMQ::bindExchange() +{ + std::atomic binding_created = false; + + /// Bridge exchange connects client's exchange with consumers' queues. + if (exchange_type == AMQP::ExchangeType::headers) + { + setup_channel->bindExchange(exchange_name, bridge_exchange, routing_keys[0], bind_headers) + .onSuccess([&]() + { + binding_created = true; + }) + .onError([&](const char * message) + { + throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + }); + } + else if (exchange_type == AMQP::ExchangeType::fanout || exchange_type == AMQP::ExchangeType::consistent_hash) + { + setup_channel->bindExchange(exchange_name, bridge_exchange, routing_keys[0]) + .onSuccess([&]() + { + binding_created = true; + }) + .onError([&](const char * message) + { + throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + }); + } + else + { + for (const auto & routing_key : routing_keys) + { + setup_channel->bindExchange(exchange_name, bridge_exchange, routing_key) + .onSuccess([&]() + { + binding_created = true; + }) + .onError([&](const char * message) + { + throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + }); + } + } + + while (!binding_created) + { + event_handler->iterateLoop(); + } +} + + +void StorageRabbitMQ::unbindExchange() +{ + if (bridge.try_lock()) + { + if (exchange_removed.load()) + return; + + setup_channel->removeExchange(bridge_exchange) + .onSuccess([&]() + { + exchange_removed.store(true); + }) + .onError([&](const char * message) + { + throw Exception("Unable to remove exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + }); + + while (!exchange_removed) + { + event_handler->iterateLoop(); + } + + event_handler->stop(); + looping_task->deactivate(); + + bridge.unlock(); + } +} + + Pipes StorageRabbitMQ::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -207,6 +345,10 @@ BlockOutputStreamPtr StorageRabbitMQ::write(const ASTPtr &, const StorageMetadat void StorageRabbitMQ::startup() { + setup_channel = std::make_shared(connection.get()); + initExchange(); + bindExchange(); + for (size_t i = 0; i < num_consumers; ++i) { try @@ -288,9 +430,9 @@ ConsumerBufferPtr StorageRabbitMQ::createReadBuffer() ChannelPtr consumer_channel = std::make_shared(connection.get()); return std::make_shared( - consumer_channel, event_handler, exchange_name, exchange_type, routing_keys, + consumer_channel, setup_channel, event_handler, consumer_exchange, exchange_type, routing_keys, next_channel_id, log, row_delimiter, hash_exchange, num_queues, - local_exchange_name, stream_cancelled); + local_exchange, stream_cancelled); } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 4457c5ff8c9..07b24e8ca1d 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -54,6 +54,9 @@ public: const String & getFormatName() const { return format_name; } NamesAndTypesList getVirtuals() const override; + const String getExchange() const { return exchange_name; } + bool checkBridge() const { return !exchange_removed.load(); } + void unbindExchange(); protected: StorageRabbitMQ( @@ -77,7 +80,6 @@ private: Names routing_keys; const String exchange_name; AMQP::ExchangeType exchange_type; - String local_exchange_name; const String format_name; char row_delimiter; @@ -99,10 +101,13 @@ private: std::mutex mutex; std::vector buffers; /// available buffers for RabbitMQ consumers - bool exchange_type_set = false; + String local_exchange, bridge_exchange, consumer_exchange; + std::mutex bridge; + AMQP::Table bind_headers; size_t next_channel_id = 1; /// Must >= 1 because it is used as a binding key, which has to be > 0 bool update_channel_id = false; - std::atomic loop_started = false; + std::atomic loop_started = false, exchange_removed = false; + ChannelPtr setup_channel; BackgroundSchedulePool::TaskHolder streaming_task; BackgroundSchedulePool::TaskHolder heartbeat_task; @@ -115,6 +120,8 @@ private: void threadFunc(); void heartbeatFunc(); void loopingFunc(); + void initExchange(); + void bindExchange(); void pingConnection() { connection->heartbeat(); } bool streamToViews(); diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 11b13714448..27e4a7b8a03 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -171,15 +171,14 @@ void WriteBufferToRabbitMQProducer::initExchange() { std::atomic exchange_declared = false, exchange_error = false; - producer_channel->declareExchange(exchange_name, exchange_type) + producer_channel->declareExchange(exchange_name, exchange_type, AMQP::durable + AMQP::passive) .onSuccess([&]() { exchange_declared = true; }) - .onError([&](const char * message) + .onError([&](const char * /* message */) { exchange_error = true; - LOG_ERROR(log, "Exchange error: {}", message); }); /// These variables are updated in a separate thread. diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index eaaa8613b5f..104ffa4e5cb 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -121,7 +121,7 @@ def test_rabbitmq_select_from_new_syntax_table(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_routing_key_list = 'new', - rabbitmq_exchange_name = 'clickhouse-exchange', + rabbitmq_exchange_name = 'new', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; ''') @@ -130,20 +130,19 @@ def test_rabbitmq_select_from_new_syntax_table(rabbitmq_cluster): parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = [] for i in range(25): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='new', body=message) + channel.basic_publish(exchange='new', routing_key='new', body=message) messages = [] for i in range(25, 50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='new', body=message) + channel.basic_publish(exchange='new', routing_key='new', body=message) connection.close() @@ -160,21 +159,20 @@ def test_rabbitmq_select_from_new_syntax_table(rabbitmq_cluster): def test_rabbitmq_select_from_old_syntax_table(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq (key UInt64, value UInt64) - ENGINE = RabbitMQ('rabbitmq1:5672', 'old', 'clickhouse-exchange', 'JSONEachRow', '\\n'); + ENGINE = RabbitMQ('rabbitmq1:5672', 'old', 'old', 'JSONEachRow', '\\n'); ''') credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='old', body=message) + channel.basic_publish(exchange='old', routing_key='old', body=message) connection.close() @@ -208,7 +206,7 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_routing_key_list = 'json', - rabbitmq_exchange_name = 'clickhouse-exchange', + rabbitmq_exchange_name = 'delim1', rabbitmq_format = 'JSONEachRow' ''') @@ -216,7 +214,6 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = '' for i in range(25): @@ -224,14 +221,14 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): all_messages = [messages] for message in all_messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='json', body=message) + channel.basic_publish(exchange='delim1', routing_key='json', body=message) messages = '' for i in range(25, 50): messages += json.dumps({'key': i, 'value': i}) + '\n' all_messages = [messages] for message in all_messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='json', body=message) + channel.basic_publish(exchange='delim1', routing_key='json', body=message) result = '' while True: @@ -250,7 +247,7 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_routing_key_list = 'csv', - rabbitmq_exchange_name = 'clickhouse-exchange', + rabbitmq_exchange_name = 'delim2', rabbitmq_format = 'CSV', rabbitmq_row_delimiter = '\\n'; ''') @@ -259,14 +256,13 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = [] for i in range(50): messages.append('{i}, {i}'.format(i=i)) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='csv', body=message) + channel.basic_publish(exchange='delim2', routing_key='csv', body=message) result = '' while True: @@ -286,7 +282,7 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_routing_key_list = 'tsv', - rabbitmq_exchange_name = 'clickhouse-exchange', + rabbitmq_exchange_name = 'delim3', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; ''') @@ -295,14 +291,13 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = [] for i in range(50): messages.append('{i}\t{i}'.format(i=i)) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='tsv', body=message) + channel.basic_publish(exchange='delim3', routing_key='tsv', body=message) result = '' while True: @@ -322,6 +317,7 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'mv', rabbitmq_routing_key_list = 'mv', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -341,7 +337,7 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='mv', body=message) + channel.basic_publish(exchange='mv', routing_key='mv', body=message) while True: result = instance.query('SELECT * FROM test.view') @@ -365,6 +361,7 @@ def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'mvsq', rabbitmq_routing_key_list = 'mvsq', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -384,7 +381,7 @@ def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='mvsq', body=message) + channel.basic_publish(exchange='mvsq', routing_key='mvsq', body=message) while True: result = instance.query('SELECT * FROM test.view') @@ -410,6 +407,7 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'mmv', rabbitmq_routing_key_list = 'mmv', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -434,7 +432,7 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='mmv', body=message) + channel.basic_publish(exchange='mmv', routing_key='mmv', body=message) while True: result1 = instance.query('SELECT * FROM test.view1') @@ -471,6 +469,7 @@ def test_rabbitmq_big_message(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value String) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'big', rabbitmq_routing_key_list = 'big', rabbitmq_format = 'JSONEachRow'; CREATE TABLE test.view (key UInt64, value String) @@ -481,7 +480,7 @@ def test_rabbitmq_big_message(rabbitmq_cluster): ''') for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key='big', body=message) + channel.basic_publish(exchange='big', routing_key='big', body=message) while True: result = instance.query('SELECT count() FROM test.view') @@ -506,6 +505,7 @@ def test_rabbitmq_sharding_between_channels_publish(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'test_channels_sharding', rabbitmq_num_consumers = 5, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -528,7 +528,6 @@ def test_rabbitmq_sharding_between_channels_publish(rabbitmq_cluster): def produce(): connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = [] for _ in range(messages_num): @@ -536,7 +535,7 @@ def test_rabbitmq_sharding_between_channels_publish(rabbitmq_cluster): i[0] += 1 key = str(randrange(1, NUM_CHANNELS)) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key=key, body=message) + channel.basic_publish(exchange='test_channels_sharding', routing_key=key, body=message) connection.close() threads = [] @@ -569,6 +568,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'test_queues_sharding', rabbitmq_num_queues = 4, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -591,7 +591,6 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): def produce(): connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = [] for _ in range(messages_num): @@ -599,7 +598,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): i[0] += 1 key = str(randrange(1, NUM_QUEUES)) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key=key, body=message) + channel.basic_publish(exchange='test_queues_sharding', routing_key=key, body=message) connection.close() threads = [] @@ -633,6 +632,7 @@ def test_rabbitmq_sharding_between_channels_and_queues_publish(rabbitmq_cluster) CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'test_sharding', rabbitmq_num_queues = 2, rabbitmq_num_consumers = 10, rabbitmq_format = 'JSONEachRow', @@ -657,7 +657,6 @@ def test_rabbitmq_sharding_between_channels_and_queues_publish(rabbitmq_cluster) def produce(): connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = [] for _ in range(messages_num): @@ -665,7 +664,7 @@ def test_rabbitmq_sharding_between_channels_and_queues_publish(rabbitmq_cluster) i[0] += 1 key = str(randrange(1, NUM_QUEUES * NUM_CONSUMERS)) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key=key, body=message) + channel.basic_publish(exchange='test_sharding', routing_key=key, body=message) connection.close() threads = [] @@ -699,6 +698,7 @@ def test_rabbitmq_read_only_combo(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'combo', rabbitmq_num_consumers = 4, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -728,7 +728,6 @@ def test_rabbitmq_read_only_combo(rabbitmq_cluster): def produce(): connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='clickhouse-exchange', exchange_type='fanout') messages = [] for _ in range(messages_num): @@ -736,7 +735,7 @@ def test_rabbitmq_read_only_combo(rabbitmq_cluster): i[0] += 1 key = str(randrange(1, NUM_CONSUMERS)) for message in messages: - channel.basic_publish(exchange='clickhouse-exchange', routing_key=key, body=message) + channel.basic_publish(exchange='combo', routing_key=key, body=message) connection.close() threads = [] @@ -787,7 +786,6 @@ def test_rabbitmq_insert(rabbitmq_cluster): consumer_connection = pika.BlockingConnection(parameters) consumer = consumer_connection.channel() - consumer.exchange_declare(exchange='insert', exchange_type='direct') result = consumer.queue_declare(queue='') queue_name = result.method.queue consumer.queue_bind(exchange='insert', queue=queue_name, routing_key='insert1') @@ -840,7 +838,6 @@ def test_rabbitmq_insert_headers_exchange(rabbitmq_cluster): consumer_connection = pika.BlockingConnection(parameters) consumer = consumer_connection.channel() - consumer.exchange_declare(exchange='insert_headers', exchange_type='headers') result = consumer.queue_declare(queue='') queue_name = result.method.queue consumer.queue_bind(exchange='insert_headers', queue=queue_name, routing_key="", @@ -890,12 +887,20 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): rabbitmq_routing_key_list = 'insert2', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; + CREATE TABLE test.rabbitmq_consume (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'many_inserts', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'insert2', + rabbitmq_format = 'TSV', + rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view_many (key UInt64, value UInt64) ENGINE = MergeTree ORDER BY key SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3; CREATE MATERIALIZED VIEW test.consumer_many TO test.view_many AS - SELECT * FROM test.rabbitmq_many; + SELECT * FROM test.rabbitmq_consume; ''') messages_num = 1000 @@ -933,6 +938,7 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): DROP TABLE IF EXISTS test.rabbitmq_many; DROP TABLE IF EXISTS test.consumer_many; DROP TABLE IF EXISTS test.view_many; + DROP TABLE IF EXISTS test.view_consume; ''') for thread in threads: @@ -946,13 +952,21 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.view_overload; DROP TABLE IF EXISTS test.consumer_overload; + CREATE TABLE test.rabbitmq_consume (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'over', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'over', + rabbitmq_num_consumers = 6, + rabbitmq_format = 'TSV', + rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.rabbitmq_overload (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'over', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'over', - rabbitmq_num_consumers = 10, rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view_overload (key UInt64, value UInt64) @@ -960,7 +974,7 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): ORDER BY key SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3; CREATE MATERIALIZED VIEW test.consumer_overload TO test.view_overload AS - SELECT * FROM test.rabbitmq_overload; + SELECT * FROM test.rabbitmq_consume; ''') messages_num = 100000 @@ -999,6 +1013,7 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): DROP TABLE IF EXISTS test.rabbitmq_overload; DROP TABLE IF EXISTS test.consumer_overload; DROP TABLE IF EXISTS test.view_overload; + DROP TABLE IF EXISTS test.view_consume; ''') for thread in threads: @@ -1044,7 +1059,6 @@ def test_rabbitmq_direct_exchange(rabbitmq_cluster): parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='direct_exchange_testing', exchange_type='direct') messages = [] for _ in range(messages_num): @@ -1118,7 +1132,6 @@ def test_rabbitmq_fanout_exchange(rabbitmq_cluster): parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='fanout_exchange_testing', exchange_type='fanout') messages = [] for _ in range(messages_num): @@ -1207,7 +1220,6 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='topic_exchange_testing', exchange_type='topic') messages = [] for _ in range(messages_num): @@ -1253,8 +1265,7 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): def test_rabbitmq_hash_exchange(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, - _consumed_by LowCardinality(String)) + CREATE TABLE test.destination(key UInt64, value UInt64, consumer_tag String) ENGINE = MergeTree() ORDER BY key; ''') @@ -1275,7 +1286,7 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.{0}_mv TO test.destination AS - SELECT key, value, '{0}' as _consumed_by FROM test.{0}; + SELECT key, value, _consumer_tag AS consumer_tag FROM test.{0}; '''.format(table_name)) i = [0] @@ -1288,14 +1299,16 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): # init connection here because otherwise python rabbitmq client might fail connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='hash_exchange_testing', exchange_type='x-consistent-hash') messages = [] for _ in range(messages_num): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 + current = 0 for message in messages: - key = str(randrange(10)) - channel.basic_publish(exchange='hash_exchange_testing', routing_key=key, body=message) + current += 1 + mes_id = str(current) + channel.basic_publish(exchange='hash_exchange_testing', routing_key=mes_id, + properties=pika.BasicProperties(message_id=mes_id), body=message) connection.close() threads = [] @@ -1307,11 +1320,13 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): time.sleep(random.uniform(0, 1)) thread.start() + result1 = '' while True: - result = instance.query('SELECT count() FROM test.destination') + result1 = instance.query('SELECT count() FROM test.destination') time.sleep(1) - if int(result) == messages_num * threads_num: + if int(result1) == messages_num * threads_num: break + result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.destination") for consumer_id in range(num_tables): table_name = 'rabbitmq_consumer{}'.format(consumer_id) @@ -1327,7 +1342,9 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): for thread in threads: thread.join() - assert int(result) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result2) >= 30 + @pytest.mark.timeout(420) @@ -1383,7 +1400,6 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): # init connection here because otherwise python rabbitmq client might fail connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='multiple_bindings_testing', exchange_type='direct') messages = [] for _ in range(messages_num): @@ -1481,7 +1497,6 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='headers_exchange_testing', exchange_type='headers') messages = [] for _ in range(messages_num): @@ -1524,20 +1539,19 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): def test_rabbitmq_virtual_columns(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.view; - CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + CREATE TABLE test.rabbitmq_virtuals (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'virtuals', rabbitmq_format = 'JSONEachRow'; CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT value, key, _exchange_name, _consumer_tag, _delivery_tag, _redelivered FROM test.rabbitmq; + SELECT value, key, _exchange_name, _consumer_tag, _delivery_tag, _redelivered FROM test.rabbitmq_virtuals; ''') credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='virtuals', exchange_type='fanout') message_num = 10 i = [0] @@ -1581,6 +1595,9 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): 8 8 virtuals amq.ctag 9 0 9 9 virtuals amq.ctag 10 0 ''' + instance.query(''' + DROP TABLE IF EXISTS test.rabbitmq_virtuals_mv + ''') assert TSV(result) == TSV(expected) @@ -1589,7 +1606,7 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + CREATE TABLE test.rabbitmq_virtuals_mv (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'virtuals_mv', @@ -1599,14 +1616,13 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): ORDER BY key; CREATE MATERIALIZED VIEW test.consumer TO test.view AS SELECT *, _exchange_name as exchange_name, _consumer_tag as consumer_tag, _delivery_tag as delivery_tag, _redelivered as redelivered - FROM test.rabbitmq; + FROM test.rabbitmq_virtuals_mv; ''') credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() - channel.exchange_declare(exchange='virtuals_mv', exchange_type='fanout') message_num = 10 i = [0] @@ -1647,8 +1663,9 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): ''' instance.query(''' - DROP TABLE test.consumer; - DROP TABLE test.view; + DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.rabbitmq_virtuals_mv ''') assert TSV(result) == TSV(expected) From f9a4bf9e6156820646cd008863af0f8a4f193c5e Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 Jul 2020 11:45:01 +0000 Subject: [PATCH 0018/1911] Add queue resume read --- .../RabbitMQ/RabbitMQBlockOutputStream.cpp | 1 + src/Storages/RabbitMQ/RabbitMQSettings.h | 1 + .../ReadBufferFromRabbitMQConsumer.cpp | 26 +++-- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 3 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 22 ++++- src/Storages/RabbitMQ/StorageRabbitMQ.h | 4 +- .../integration/test_storage_rabbitmq/test.py | 94 ++++++++++++++++++- 7 files changed, 137 insertions(+), 14 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index 87a17d3e1ed..c2eae19cb86 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -35,6 +35,7 @@ void RabbitMQBlockOutputStream::writePrefix() { if (storage.checkBridge()) storage.unbindExchange(); + buffer = storage.createWriteBuffer(); if (!buffer) throw Exception("Failed to create RabbitMQ producer!", ErrorCodes::CANNOT_CREATE_IO_BUFFER); diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index 5cd52ed9ef7..488fc59e562 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -19,6 +19,7 @@ namespace DB M(SettingUInt64, rabbitmq_num_consumers, 1, "The number of consumer channels per table.", 0) \ M(SettingUInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \ M(SettingBool, rabbitmq_transactional_channel, false, "Use transactional channel for publishing.", 0) \ + M(SettingString, rabbitmq_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \ DECLARE_SETTINGS_COLLECTION(LIST_OF_RABBITMQ_SETTINGS) diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 8c272e04691..5abdb4fe7c2 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -24,6 +24,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( const AMQP::ExchangeType & exchange_type_, const Names & routing_keys_, size_t channel_id_, + const String & queue_base_, Poco::Logger * log_, char row_delimiter_, bool hash_exchange_, @@ -38,6 +39,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , exchange_type(exchange_type_) , routing_keys(routing_keys_) , channel_id(channel_id_) + , queue_base(queue_base_) , hash_exchange(hash_exchange_) , num_queues(num_queues_) , log(log_) @@ -54,7 +56,6 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() { consumer_channel->close(); - received.clear(); BufferBase::set(nullptr, 0, 0); } @@ -64,12 +65,14 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) { bool bindings_created = false, bindings_error = false; - setup_channel->declareQueue(AMQP::exclusive) - .onSuccess([&](const std::string & queue_name_, int /* msgcount */, int /* consumercount */) + auto success_callback = [&](const std::string & queue_name_, int msgcount, int /* consumercount */) { queues.emplace_back(queue_name_); LOG_DEBUG(log, "Queue " + queue_name_ + " is declared"); + if (msgcount) + LOG_TRACE(log, "Queue " + queue_name_ + " is non-empty. Non-consumed messaged will also be delivered."); + subscribed_queue[queue_name_] = false; subscribe(queues.back()); @@ -86,7 +89,6 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) */ String current_hash_exchange = exchange_type == AMQP::ExchangeType::consistent_hash ? exchange_name : local_exchange; - /// If hash-exchange is used for messages distribution, then the binding key is ignored - can be arbitrary. setup_channel->bindQueue(current_hash_exchange, queue_name_, binding_key) .onSuccess([&] { @@ -116,7 +118,6 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) AMQP::Table binding_arguments; std::vector matching; - /// It is not parsed for the second time - if it was parsed above, then it would never end up here. for (const auto & header : routing_keys) { boost::split(matching, header, [](char c){ return c == '='; }); @@ -153,13 +154,24 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) }); } } - }) - .onError([&](const char * message) + }; + + auto error_callback([&](const char * message) { bindings_error = true; LOG_ERROR(log, "Failed to declare queue on the channel. Reason: {}", message); }); + if (!queue_base.empty()) + { + const String queue_name = !hash_exchange ? queue_base : queue_base + "_" + std::to_string(channel_id) + "_" + std::to_string(queue_id); + setup_channel->declareQueue(queue_name, AMQP::durable).onSuccess(success_callback).onError(error_callback); + } + else + { + setup_channel->declareQueue(AMQP::durable).onSuccess(success_callback).onError(error_callback); + } + while (!bindings_created && !bindings_error) { iterateEventLoop(); diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 6896dd7f4b0..f4978e54229 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -30,6 +30,7 @@ public: const AMQP::ExchangeType & exchange_type_, const Names & routing_keys_, size_t channel_id_, + const String & queue_base_, Poco::Logger * log_, char row_delimiter_, bool hash_exchange_, @@ -62,6 +63,7 @@ private: const AMQP::ExchangeType exchange_type; const Names routing_keys; const size_t channel_id; + const String queue_base; const bool hash_exchange; const size_t num_queues; @@ -82,7 +84,6 @@ private: bool nextImpl() override; - void connectAlternateExchange(); void initQueueBindings(const size_t queue_id); void subscribe(const String & queue_name); void iterateEventLoop(); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index d56a46c4f55..db4f1c7b338 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -72,7 +72,8 @@ StorageRabbitMQ::StorageRabbitMQ( const String & exchange_type_, size_t num_consumers_, size_t num_queues_, - const bool use_transactional_channel_) + const bool use_transactional_channel_, + const String & queue_base_) : IStorage(table_id_) , global_context(context_.getGlobalContext()) , rabbitmq_context(Context(global_context)) @@ -83,6 +84,7 @@ StorageRabbitMQ::StorageRabbitMQ( , num_consumers(num_consumers_) , num_queues(num_queues_) , use_transactional_channel(use_transactional_channel_) + , queue_base(queue_base_) , log(&Poco::Logger::get("StorageRabbitMQ (" + table_id_.table_name + ")")) , parsed_address(parseAddress(global_context.getMacros()->expand(host_port_), 5672)) , login_password(std::make_pair( @@ -288,7 +290,7 @@ void StorageRabbitMQ::unbindExchange() throw Exception("Unable to remove exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); }); - while (!exchange_removed) + while (!exchange_removed.load()) { event_handler->iterateLoop(); } @@ -431,7 +433,7 @@ ConsumerBufferPtr StorageRabbitMQ::createReadBuffer() return std::make_shared( consumer_channel, setup_channel, event_handler, consumer_exchange, exchange_type, routing_keys, - next_channel_id, log, row_delimiter, hash_exchange, num_queues, + next_channel_id, queue_base, log, row_delimiter, hash_exchange, num_queues, local_exchange, stream_cancelled); } @@ -725,10 +727,22 @@ void registerStorageRabbitMQ(StorageFactory & factory) } } + String queue_base = rabbitmq_settings.rabbitmq_queue_base.value; + if (args_count >= 10) + { + engine_args[9] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[9], args.local_context); + + const auto * ast = engine_args[9]->as(); + if (ast && ast->value.getType() == Field::Types::String) + { + queue_base = safeGet(ast->value); + } + } + return StorageRabbitMQ::create( args.table_id, args.context, args.columns, host_port, routing_keys, exchange, format, row_delimiter, exchange_type, num_consumers, - num_queues, use_transactional_channel); + num_queues, use_transactional_channel, queue_base); }; factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 07b24e8ca1d..d43f2ba27f1 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -71,7 +71,8 @@ protected: const String & exchange_type_, size_t num_consumers_, size_t num_queues_, - const bool use_transactional_channel_); + const bool use_transactional_channel_, + const String & queue_base_); private: Context global_context; @@ -88,6 +89,7 @@ private: bool hash_exchange; size_t num_queues; const bool use_transactional_channel; + const String queue_base; Poco::Logger * log; std::pair parsed_address; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 104ffa4e5cb..655dee7a816 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -877,6 +877,7 @@ def test_rabbitmq_insert_headers_exchange(rabbitmq_cluster): def test_rabbitmq_many_inserts(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.rabbitmq_many; + DROP TABLE IF EXISTS test.rabbitmq_consume; DROP TABLE IF EXISTS test.view_many; DROP TABLE IF EXISTS test.consumer_many; CREATE TABLE test.rabbitmq_many (key UInt64, value UInt64) @@ -935,10 +936,10 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): break instance.query(''' + DROP TABLE IF EXISTS test.rabbitmq_consume; DROP TABLE IF EXISTS test.rabbitmq_many; DROP TABLE IF EXISTS test.consumer_many; DROP TABLE IF EXISTS test.view_many; - DROP TABLE IF EXISTS test.view_consume; ''') for thread in threads: @@ -952,6 +953,7 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.view_overload; DROP TABLE IF EXISTS test.consumer_overload; + DROP TABLE IF EXISTS test.rabbitmq_consume; CREATE TABLE test.rabbitmq_consume (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', @@ -1671,6 +1673,96 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): assert TSV(result) == TSV(expected) +@pytest.mark.timeout(420) +def test_rabbitmq_queue_resume(rabbitmq_cluster): + instance.query(''' + CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'queue_resume', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'queue_resume', + rabbitmq_queue_base = 'queue_resume', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.view (key UInt64, value UInt64, consumer_tag String) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT *, _consumer_tag AS consumer_tag FROM test.rabbitmq_queue_resume; + ''') + + i = [0] + messages_num = 5000 + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + + def produce(): + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + messages = [] + for _ in range(messages_num): + messages.append(json.dumps({'key': i[0], 'value': i[0]})) + i[0] += 1 + for message in messages: + channel.basic_publish(exchange='queue_resume', routing_key='queue_resume', body=message, + properties=pika.BasicProperties(delivery_mode = 2)) + connection.close() + + threads = [] + threads_num = 10 + for _ in range(threads_num): + threads.append(threading.Thread(target=produce)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + while int(instance.query('SELECT count() FROM test.view')) == 0: + time.sleep(1) + + instance.query(''' + DROP TABLE IF EXISTS test.rabbitmq_queue_resume; + ''') + + for thread in threads: + thread.join() + + collected = int(instance.query('SELECT count() FROM test.view')) + + instance.query(''' + CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'queue_resume', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'queue_resume', + rabbitmq_queue_base = 'queue_resume', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + ''') + + while True: + result1 = instance.query('SELECT count() FROM test.view') + time.sleep(1) + if int(result1) > collected: + break + + result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") + + instance.query(''' + DROP TABLE IF EXISTS test.rabbitmq_queue_resume; + DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.view; + ''') + + assert int(result1) > collected, 'ClickHouse lost some messages: {}'.format(result) + assert int(result2) == 2 + + + if __name__ == '__main__': cluster.start() raw_input("Cluster created, press any key to destroy...") From 2b57857afc7e220f7844f98ef26fb5aff24ed6c4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Jul 2020 12:33:07 +0000 Subject: [PATCH 0019/1911] Add dl-exchange, commits --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 11 + .../RabbitMQ/RabbitMQBlockInputStream.h | 2 + src/Storages/RabbitMQ/RabbitMQHandler.cpp | 2 - src/Storages/RabbitMQ/RabbitMQHandler.h | 3 +- src/Storages/RabbitMQ/RabbitMQSettings.h | 1 + .../ReadBufferFromRabbitMQConsumer.cpp | 50 ++- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 8 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 25 +- src/Storages/RabbitMQ/StorageRabbitMQ.h | 4 +- .../integration/test_storage_rabbitmq/test.py | 353 ++++++++++-------- 10 files changed, 296 insertions(+), 163 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index 6e8e153392c..630581b13dc 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -137,6 +137,8 @@ Block RabbitMQBlockInputStream::readImpl() virtual_columns[3]->insert(redelivered); } + last_inserted_delivery_tag = delivery_tag; + total_rows = total_rows + new_rows; buffer->allowNext(); @@ -158,4 +160,13 @@ Block RabbitMQBlockInputStream::readImpl() return result_block; } + +void RabbitMQBlockInputStream::readSuffixImpl() +{ + if (!buffer) + return; + + buffer->ackMessages(last_inserted_delivery_tag); +} + } diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h index f4ab76f72cf..09cda6ff94f 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h @@ -26,6 +26,7 @@ public: void readPrefixImpl() override; Block readImpl() override; + void readSuffixImpl() override; private: StorageRabbitMQ & storage; @@ -38,6 +39,7 @@ private: const Block virtual_header; ConsumerBufferPtr buffer; + UInt64 last_inserted_delivery_tag; }; } diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.cpp b/src/Storages/RabbitMQ/RabbitMQHandler.cpp index f01b1e60eab..5d17ff23b64 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.cpp +++ b/src/Storages/RabbitMQ/RabbitMQHandler.cpp @@ -31,11 +31,9 @@ void RabbitMQHandler::onError(AMQP::TcpConnection * connection, const char * mes void RabbitMQHandler::startLoop() { std::lock_guard lock(startup_mutex); - loop_started.store(true); /// stop_loop variable is updated in a separate thread while (!stop_loop.load()) uv_run(loop, UV_RUN_NOWAIT); - loop_started.store(false); } void RabbitMQHandler::iterateLoop() diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.h b/src/Storages/RabbitMQ/RabbitMQHandler.h index b1b84e1d07a..5893ace1d2f 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.h +++ b/src/Storages/RabbitMQ/RabbitMQHandler.h @@ -21,13 +21,12 @@ public: void stop() { stop_loop.store(true); } void startLoop(); void iterateLoop(); - bool checkLoop() const { return loop_started.load(); } private: uv_loop_t * loop; Poco::Logger * log; - std::atomic stop_loop = false, loop_started = false; + std::atomic stop_loop = false; std::mutex startup_mutex; }; diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index 488fc59e562..cd7e7de9622 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -20,6 +20,7 @@ namespace DB M(SettingUInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \ M(SettingBool, rabbitmq_transactional_channel, false, "Use transactional channel for publishing.", 0) \ M(SettingString, rabbitmq_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \ + M(SettingString, rabbitmq_deadletter_exchange, "", "Exchange name to be passed as a dead-letter-exchange name.", 0) \ DECLARE_SETTINGS_COLLECTION(LIST_OF_RABBITMQ_SETTINGS) diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 5abdb4fe7c2..705aae7ec61 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -30,6 +30,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( bool hash_exchange_, size_t num_queues_, const String & local_exchange_, + const String & deadletter_exchange_, const std::atomic & stopped_) : ReadBuffer(nullptr, 0) , consumer_channel(std::move(consumer_channel_)) @@ -46,6 +47,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , row_delimiter(row_delimiter_) , stopped(stopped_) , local_exchange(local_exchange_) + , deadletter_exchange(deadletter_exchange_) , received(QUEUE_SIZE * num_queues) { for (size_t queue_id = 0; queue_id < num_queues; ++queue_id) @@ -55,6 +57,12 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() { + if (ack.load() && consumer_channel) + { + consumer_channel->ack(prev_tag, AMQP::multiple); /// Will ack all up to last tag staring from last acked. + LOG_TRACE(log, "Acknowledged messages with deliveryTags up to {}", prev_tag); + } + consumer_channel->close(); received.clear(); BufferBase::set(nullptr, 0, 0); @@ -162,14 +170,20 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) LOG_ERROR(log, "Failed to declare queue on the channel. Reason: {}", message); }); + AMQP::Table queue_settings; + if (!deadletter_exchange.empty()) + { + queue_settings["x-dead-letter-exchange"] = deadletter_exchange; + } + if (!queue_base.empty()) { const String queue_name = !hash_exchange ? queue_base : queue_base + "_" + std::to_string(channel_id) + "_" + std::to_string(queue_id); - setup_channel->declareQueue(queue_name, AMQP::durable).onSuccess(success_callback).onError(error_callback); + setup_channel->declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); } else { - setup_channel->declareQueue(AMQP::durable).onSuccess(success_callback).onError(error_callback); + setup_channel->declareQueue(AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); } while (!bindings_created && !bindings_error) @@ -184,15 +198,20 @@ void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name) if (subscribed_queue[queue_name]) return; - consumer_channel->consume(queue_name, AMQP::noack) + consumer_channel->consume(queue_name) .onSuccess([&](const std::string & consumer) { subscribed_queue[queue_name] = true; - consumer_error = false; ++count_subscribed; + LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); + + consumer_error = false; consumer_tag = consumer; - LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); + consumer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Consumer {} error: {}", consumer_tag, message); + }); }) .onReceived([&](const AMQP::Message & message, uint64_t deliveryTag, bool redelivered) { @@ -201,11 +220,16 @@ void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name) { String message_received = std::string(message.body(), message.body() + message_size); if (row_delimiter != '\0') - { message_received += row_delimiter; - } received.push({deliveryTag, message_received, redelivered}); + + std::lock_guard lock(wait_ack); + if (ack.exchange(false) && prev_tag < max_tag && consumer_channel) + { + consumer_channel->ack(prev_tag, AMQP::multiple); /// Will ack all up to last tag staring from last acked. + LOG_TRACE(log, "Consumer {} acknowledged messages with deliveryTags up to {}", consumer_tag, prev_tag); + } } }) .onError([&](const char * message) @@ -243,6 +267,17 @@ void ReadBufferFromRabbitMQConsumer::checkSubscription() } +void ReadBufferFromRabbitMQConsumer::ackMessages(UInt64 last_inserted_delivery_tag) +{ + if (last_inserted_delivery_tag > prev_tag) + { + std::lock_guard lock(wait_ack); + prev_tag = last_inserted_delivery_tag; + ack.store(true); + } +} + + void ReadBufferFromRabbitMQConsumer::iterateEventLoop() { event_handler->iterateLoop(); @@ -259,6 +294,7 @@ bool ReadBufferFromRabbitMQConsumer::nextImpl() auto * new_position = const_cast(current.message.data()); BufferBase::set(new_position, current.message.size(), 0); allowed = false; + max_tag = current.delivery_tag; return true; } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index f4978e54229..8033f537e8c 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -36,6 +36,7 @@ public: bool hash_exchange_, size_t num_queues_, const String & local_exchange_, + const String & deadletter_exchange_, const std::atomic & stopped_); ~ReadBufferFromRabbitMQConsumer() override; @@ -49,6 +50,7 @@ public: void allowNext() { allowed = true; } // Allow to read next message. void checkSubscription(); + void ackMessages(UInt64 last_inserted_delivery_tag); auto getConsumerTag() const { return consumer_tag; } auto getDeliveryTag() const { return current.delivery_tag; } @@ -72,15 +74,19 @@ private: bool allowed = true; const std::atomic & stopped; - const String local_exchange; + const String local_exchange, deadletter_exchange; std::atomic consumer_error = false; std::atomic count_subscribed = 0, wait_subscribed; String consumer_tag; ConcurrentBoundedQueue received; + UInt64 prev_tag = 0; MessageData current; std::vector queues; std::unordered_map subscribed_queue; + std::atomic ack = false; + std::mutex wait_ack; + UInt64 max_tag = 0; bool nextImpl() override; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index db4f1c7b338..52a07026c24 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -73,7 +73,8 @@ StorageRabbitMQ::StorageRabbitMQ( size_t num_consumers_, size_t num_queues_, const bool use_transactional_channel_, - const String & queue_base_) + const String & queue_base_, + const String & deadletter_exchange_) : IStorage(table_id_) , global_context(context_.getGlobalContext()) , rabbitmq_context(Context(global_context)) @@ -85,6 +86,7 @@ StorageRabbitMQ::StorageRabbitMQ( , num_queues(num_queues_) , use_transactional_channel(use_transactional_channel_) , queue_base(queue_base_) + , deadletter_exchange(deadletter_exchange_) , log(&Poco::Logger::get("StorageRabbitMQ (" + table_id_.table_name + ")")) , parsed_address(parseAddress(global_context.getMacros()->expand(host_port_), 5672)) , login_password(std::make_pair( @@ -224,6 +226,7 @@ void StorageRabbitMQ::initExchange() void StorageRabbitMQ::bindExchange() { std::atomic binding_created = false; + size_t bound_keys = 0; /// Bridge exchange connects client's exchange with consumers' queues. if (exchange_type == AMQP::ExchangeType::headers) @@ -257,7 +260,9 @@ void StorageRabbitMQ::bindExchange() setup_channel->bindExchange(exchange_name, bridge_exchange, routing_key) .onSuccess([&]() { - binding_created = true; + ++bound_keys; + if (bound_keys == routing_keys.size()) + binding_created = true; }) .onError([&](const char * message) { @@ -434,7 +439,7 @@ ConsumerBufferPtr StorageRabbitMQ::createReadBuffer() return std::make_shared( consumer_channel, setup_channel, event_handler, consumer_exchange, exchange_type, routing_keys, next_channel_id, queue_base, log, row_delimiter, hash_exchange, num_queues, - local_exchange, stream_cancelled); + local_exchange, deadletter_exchange, stream_cancelled); } @@ -739,10 +744,22 @@ void registerStorageRabbitMQ(StorageFactory & factory) } } + String deadletter_exchange = rabbitmq_settings.rabbitmq_deadletter_exchange.value; + if (args_count >= 11) + { + engine_args[10] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[10], args.local_context); + + const auto * ast = engine_args[9]->as(); + if (ast && ast->value.getType() == Field::Types::String) + { + deadletter_exchange = safeGet(ast->value); + } + } + return StorageRabbitMQ::create( args.table_id, args.context, args.columns, host_port, routing_keys, exchange, format, row_delimiter, exchange_type, num_consumers, - num_queues, use_transactional_channel, queue_base); + num_queues, use_transactional_channel, queue_base, deadletter_exchange); }; factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index d43f2ba27f1..7e2d6c6b35e 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -72,7 +72,8 @@ protected: size_t num_consumers_, size_t num_queues_, const bool use_transactional_channel_, - const String & queue_base_); + const String & queue_base_, + const String & deadletter_exchange); private: Context global_context; @@ -90,6 +91,7 @@ private: size_t num_queues; const bool use_transactional_channel; const String queue_base; + const String deadletter_exchange; Poco::Logger * log; std::pair parsed_address; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 655dee7a816..3a2b6cd6be3 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -496,135 +496,9 @@ def test_rabbitmq_big_message(rabbitmq_cluster): assert int(result) == rabbitmq_messages*batch_messages, 'ClickHouse lost some messages: {}'.format(result) -@pytest.mark.timeout(420) -def test_rabbitmq_sharding_between_channels_publish(rabbitmq_cluster): - - NUM_CHANNELS = 5 - - instance.query(''' - CREATE TABLE test.rabbitmq (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'test_channels_sharding', - rabbitmq_num_consumers = 5, - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.rabbitmq; - ''') - - time.sleep(1) - - i = [0] - messages_num = 10000 - - credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) - def produce(): - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({'key': i[0], 'value': i[0]})) - i[0] += 1 - key = str(randrange(1, NUM_CHANNELS)) - for message in messages: - channel.basic_publish(exchange='test_channels_sharding', routing_key=key, body=message) - connection.close() - - threads = [] - threads_num = 20 - - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - while True: - result = instance.query('SELECT count() FROM test.view') - time.sleep(1) - if int(result) == messages_num * threads_num: - break - - for thread in threads: - thread.join() - - assert int(result) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) - - @pytest.mark.timeout(420) def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): - NUM_QUEUES = 4 - - instance.query(''' - CREATE TABLE test.rabbitmq (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'test_queues_sharding', - rabbitmq_num_queues = 4, - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.rabbitmq; - ''') - - time.sleep(1) - - i = [0] - messages_num = 10000 - - credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) - def produce(): - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({'key': i[0], 'value': i[0]})) - i[0] += 1 - key = str(randrange(1, NUM_QUEUES)) - for message in messages: - channel.basic_publish(exchange='test_queues_sharding', routing_key=key, body=message) - connection.close() - - threads = [] - threads_num = 20 - - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - while True: - result = instance.query('SELECT count() FROM test.view') - time.sleep(1) - if int(result) == messages_num * threads_num: - break - - for thread in threads: - thread.join() - - assert int(result) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) - - -@pytest.mark.timeout(420) -def test_rabbitmq_sharding_between_channels_and_queues_publish(rabbitmq_cluster): - NUM_CONSUMERS = 10 NUM_QUEUES = 2 @@ -639,12 +513,12 @@ def test_rabbitmq_sharding_between_channels_and_queues_publish(rabbitmq_cluster) rabbitmq_row_delimiter = '\\n'; DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.view (key UInt64, value UInt64) + CREATE TABLE test.view (key UInt64, value UInt64, consumer_tag String) ENGINE = MergeTree ORDER BY key SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3; CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.rabbitmq; + SELECT *, _consumer_tag AS consumer_tag FROM test.rabbitmq; ''') time.sleep(1) @@ -662,9 +536,12 @@ def test_rabbitmq_sharding_between_channels_and_queues_publish(rabbitmq_cluster) for _ in range(messages_num): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 - key = str(randrange(1, NUM_QUEUES * NUM_CONSUMERS)) + current = 0 for message in messages: - channel.basic_publish(exchange='test_sharding', routing_key=key, body=message) + current += 1 + mes_id = str(current) + channel.basic_publish(exchange='test_sharding', routing_key='', + properties=pika.BasicProperties(message_id=mes_id), body=message) connection.close() threads = [] @@ -676,16 +553,20 @@ def test_rabbitmq_sharding_between_channels_and_queues_publish(rabbitmq_cluster) time.sleep(random.uniform(0, 1)) thread.start() + result1 = '' while True: - result = instance.query('SELECT count() FROM test.view') + result1 = instance.query('SELECT count() FROM test.view') time.sleep(1) - if int(result) == messages_num * threads_num: + if int(result1) == messages_num * threads_num: break + result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") + for thread in threads: thread.join() - assert int(result) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result2) == 10 @pytest.mark.timeout(420) @@ -734,8 +615,12 @@ def test_rabbitmq_read_only_combo(rabbitmq_cluster): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 key = str(randrange(1, NUM_CONSUMERS)) + current = 0 for message in messages: - channel.basic_publish(exchange='combo', routing_key=key, body=message) + current += 1 + mes_id = str(current) + channel.basic_publish(exchange='combo', routing_key=key, + properties=pika.BasicProperties(message_id=mes_id), body=message) connection.close() threads = [] @@ -1140,11 +1025,11 @@ def test_rabbitmq_fanout_exchange(rabbitmq_cluster): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 - key_num = 0 + current = 0 for message in messages: - mes_id = str(randrange(10)) - channel.basic_publish( - exchange='fanout_exchange_testing', routing_key='', + current += 1 + mes_id = str(current) + channel.basic_publish(exchange='fanout_exchange_testing', routing_key='', properties=pika.BasicProperties(message_id=mes_id), body=message) connection.close() @@ -1236,10 +1121,11 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): channel.basic_publish(exchange='topic_exchange_testing', routing_key=key, body=message) key = "random.logs" + current = 0 for message in messages: - mes_id = str(randrange(10)) - channel.basic_publish( - exchange='topic_exchange_testing', routing_key=key, + current += 1 + mes_id = str(current) + channel.basic_publish(exchange='topic_exchange_testing', routing_key=key, properties=pika.BasicProperties(message_id=mes_id), body=message) connection.close() @@ -1411,8 +1297,10 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): keys = ['key1', 'key2', 'key3', 'key4', 'key5'] for key in keys: + current = 0 for message in messages: - mes_id = str(randrange(10)) + current += 1 + mes_id = str(current) channel.basic_publish(exchange='multiple_bindings_testing', routing_key=key, properties=pika.BasicProperties(message_id=mes_id), body=message) @@ -1510,9 +1398,10 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): fields['type']='report' fields['year']='2020' - key_num = 0 + current = 0 for message in messages: - mes_id = str(randrange(10)) + current += 1 + mes_id = str(current) channel.basic_publish(exchange='headers_exchange_testing', routing_key='', properties=pika.BasicProperties(headers=fields, message_id=mes_id), body=message) @@ -1674,7 +1563,91 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): @pytest.mark.timeout(420) -def test_rabbitmq_queue_resume(rabbitmq_cluster): +def test_rabbitmq_queue_resume_1(rabbitmq_cluster): + instance.query(''' + CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'queue_resume', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'queue_resume', + rabbitmq_num_consumers = '2', + rabbitmq_num_queues = '2', + rabbitmq_queue_base = 'queue_resume', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + ''') + + i = [0] + messages_num = 5000 + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + def produce(): + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + messages = [] + for _ in range(messages_num): + messages.append(json.dumps({'key': i[0], 'value': i[0]})) + i[0] += 1 + for message in messages: + channel.basic_publish(exchange='queue_resume', routing_key='queue_resume', body=message, + properties=pika.BasicProperties(delivery_mode = 2)) + connection.close() + + threads = [] + threads_num = 10 + for _ in range(threads_num): + threads.append(threading.Thread(target=produce)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + for thread in threads: + thread.join() + + instance.query(''' + DROP TABLE IF EXISTS test.rabbitmq_queue_resume; + ''') + + instance.query(''' + CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'queue_resume', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'queue_resume', + rabbitmq_num_consumers = '2', + rabbitmq_num_queues = '2', + rabbitmq_queue_base = 'queue_resume', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.rabbitmq_queue_resume; + ''') + + while True: + result1 = instance.query('SELECT count() FROM test.view') + time.sleep(1) + if int(result1) == messages_num * threads_num: + break + + instance.query(''' + DROP TABLE IF EXISTS test.rabbitmq_queue_resume; + DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.view; + ''') + + assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + + +@pytest.mark.timeout(420) +def test_rabbitmq_queue_resume_2(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) ENGINE = RabbitMQ @@ -1699,7 +1672,6 @@ def test_rabbitmq_queue_resume(rabbitmq_cluster): credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) - def produce(): connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1762,6 +1734,95 @@ def test_rabbitmq_queue_resume(rabbitmq_cluster): assert int(result2) == 2 +@pytest.mark.timeout(420) +def test_rabbitmq_consumer_acknowledgements(rabbitmq_cluster): + instance.query(''' + CREATE TABLE test.rabbitmq_consumer_acks (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'consumer_acks', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'consumer_acks', + rabbitmq_queue_base = 'consumer_resume', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + ''') + + i = [0] + messages_num = 5000 + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + def produce(): + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + messages = [] + for _ in range(messages_num): + messages.append(json.dumps({'key': i[0], 'value': i[0]})) + i[0] += 1 + for message in messages: + channel.basic_publish(exchange='consumer_acks', routing_key='consumer_acks', body=message, + properties=pika.BasicProperties(delivery_mode = 2)) + connection.close() + + threads = [] + threads_num = 20 + for _ in range(threads_num): + threads.append(threading.Thread(target=produce)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + for thread in threads: + thread.join() + + instance.query(''' + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.view (key UInt64, value UInt64, consumer_tag String) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT *, _consumer_tag AS consumer_tag FROM test.rabbitmq_consumer_acks; + ''') + + while int(instance.query('SELECT count() FROM test.view')) == 0: + time.sleep(1) + + instance.query(''' + DROP TABLE IF EXISTS test.rabbitmq_consumer_acks; + ''') + + collected = int(instance.query('SELECT count() FROM test.view')) + + instance.query(''' + CREATE TABLE test.rabbitmq_consumer_acks (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_queue_base = 'consumer_resume', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + ''') + + while True: + result1 = instance.query('SELECT count() FROM test.view') + time.sleep(1) + #print("receiived", result1, "collected", collected) + if int(result1) == messages_num * threads_num: + break + + result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") + + instance.query(''' + DROP TABLE IF EXISTS test.rabbitmq_consumer_acks; + DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.view; + ''') + + assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + if collected < result1: + assert int(result2) == 2 + if __name__ == '__main__': cluster.start() From 22b16060f685fbe98edc18aba68b2e517aa774bc Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 25 Jul 2020 11:14:46 +0000 Subject: [PATCH 0020/1911] More reliable publishing --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 3 +- .../RabbitMQ/RabbitMQBlockOutputStream.cpp | 3 + src/Storages/RabbitMQ/RabbitMQSettings.h | 1 + .../ReadBufferFromRabbitMQConsumer.cpp | 13 +-- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 3 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 30 ++++-- src/Storages/RabbitMQ/StorageRabbitMQ.h | 6 +- .../WriteBufferToRabbitMQProducer.cpp | 93 +++++++++++-------- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 11 ++- .../integration/test_storage_rabbitmq/test.py | 15 ++- 10 files changed, 104 insertions(+), 74 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index 630581b13dc..0c70acaf1e3 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -137,7 +137,8 @@ Block RabbitMQBlockInputStream::readImpl() virtual_columns[3]->insert(redelivered); } - last_inserted_delivery_tag = delivery_tag; + if (delivery_tag > last_inserted_delivery_tag) + last_inserted_delivery_tag = delivery_tag; total_rows = total_rows + new_rows; buffer->allowNext(); diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index c2eae19cb86..37b39bbaeae 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -61,7 +61,10 @@ void RabbitMQBlockOutputStream::writeSuffix() child->writeSuffix(); if (buffer) + { + buffer->updateMaxWait(); buffer->finilizeProducer(); + } } } diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index cd7e7de9622..0f65fe6553c 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -21,6 +21,7 @@ namespace DB M(SettingBool, rabbitmq_transactional_channel, false, "Use transactional channel for publishing.", 0) \ M(SettingString, rabbitmq_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \ M(SettingString, rabbitmq_deadletter_exchange, "", "Exchange name to be passed as a dead-letter-exchange name.", 0) \ + M(SettingBool, rabbitmq_persistent_mode, false, "If set, delivery mode will be set to 2 (makes messages 'persistent', durable).", 0) \ DECLARE_SETTINGS_COLLECTION(LIST_OF_RABBITMQ_SETTINGS) diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 705aae7ec61..b1e63005126 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -57,13 +57,10 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() { - if (ack.load() && consumer_channel) - { - consumer_channel->ack(prev_tag, AMQP::multiple); /// Will ack all up to last tag staring from last acked. - LOG_TRACE(log, "Acknowledged messages with deliveryTags up to {}", prev_tag); - } - + if (ack.load() && max_tag && consumer_channel) + consumer_channel->ack(max_tag, AMQP::multiple); consumer_channel->close(); + received.clear(); BufferBase::set(nullptr, 0, 0); } @@ -225,7 +222,7 @@ void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name) received.push({deliveryTag, message_received, redelivered}); std::lock_guard lock(wait_ack); - if (ack.exchange(false) && prev_tag < max_tag && consumer_channel) + if (ack.exchange(false) && prev_tag && prev_tag <= max_tag && consumer_channel) { consumer_channel->ack(prev_tag, AMQP::multiple); /// Will ack all up to last tag staring from last acked. LOG_TRACE(log, "Consumer {} acknowledged messages with deliveryTags up to {}", consumer_tag, prev_tag); @@ -271,7 +268,7 @@ void ReadBufferFromRabbitMQConsumer::ackMessages(UInt64 last_inserted_delivery_t { if (last_inserted_delivery_tag > prev_tag) { - std::lock_guard lock(wait_ack); + std::lock_guard lock(wait_ack); /// See onReceived() callback. prev_tag = last_inserted_delivery_tag; ack.store(true); } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 8033f537e8c..6d2deb0be03 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -80,13 +80,12 @@ private: String consumer_tag; ConcurrentBoundedQueue received; - UInt64 prev_tag = 0; + UInt64 prev_tag = 0, max_tag = 0; MessageData current; std::vector queues; std::unordered_map subscribed_queue; std::atomic ack = false; std::mutex wait_ack; - UInt64 max_tag = 0; bool nextImpl() override; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 52a07026c24..daa17719654 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -74,7 +74,8 @@ StorageRabbitMQ::StorageRabbitMQ( size_t num_queues_, const bool use_transactional_channel_, const String & queue_base_, - const String & deadletter_exchange_) + const String & deadletter_exchange_, + const bool persistent_) : IStorage(table_id_) , global_context(context_.getGlobalContext()) , rabbitmq_context(Context(global_context)) @@ -87,6 +88,7 @@ StorageRabbitMQ::StorageRabbitMQ( , use_transactional_channel(use_transactional_channel_) , queue_base(queue_base_) , deadletter_exchange(deadletter_exchange_) + , persistent(persistent_) , log(&Poco::Logger::get("StorageRabbitMQ (" + table_id_.table_name + ")")) , parsed_address(parseAddress(global_context.getMacros()->expand(host_port_), 5672)) , login_password(std::make_pair( @@ -280,7 +282,7 @@ void StorageRabbitMQ::bindExchange() void StorageRabbitMQ::unbindExchange() { - if (bridge.try_lock()) + std::call_once(flag, [&]() { if (exchange_removed.load()) return; @@ -302,9 +304,7 @@ void StorageRabbitMQ::unbindExchange() event_handler->stop(); looping_task->deactivate(); - - bridge.unlock(); - } + }); } @@ -447,7 +447,7 @@ ProducerBufferPtr StorageRabbitMQ::createWriteBuffer() { return std::make_shared( parsed_address, global_context, login_password, routing_keys, exchange_name, exchange_type, - log, num_consumers * num_queues, use_transactional_channel, + log, num_consumers * num_queues, use_transactional_channel, persistent, row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); } @@ -749,17 +749,31 @@ void registerStorageRabbitMQ(StorageFactory & factory) { engine_args[10] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[10], args.local_context); - const auto * ast = engine_args[9]->as(); + const auto * ast = engine_args[10]->as(); if (ast && ast->value.getType() == Field::Types::String) { deadletter_exchange = safeGet(ast->value); } } + bool persistent = static_cast(rabbitmq_settings.rabbitmq_persistent_mode); + if (args_count >= 12) + { + const auto * ast = engine_args[11]->as(); + if (ast && ast->value.getType() == Field::Types::UInt64) + { + persistent = static_cast(safeGet(ast->value)); + } + else + { + throw Exception("Transactional channel parameter is a bool", ErrorCodes::BAD_ARGUMENTS); + } + } + return StorageRabbitMQ::create( args.table_id, args.context, args.columns, host_port, routing_keys, exchange, format, row_delimiter, exchange_type, num_consumers, - num_queues, use_transactional_channel, queue_base, deadletter_exchange); + num_queues, use_transactional_channel, queue_base, deadletter_exchange, persistent); }; factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 7e2d6c6b35e..9c7df1b1421 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -73,7 +73,8 @@ protected: size_t num_queues_, const bool use_transactional_channel_, const String & queue_base_, - const String & deadletter_exchange); + const String & deadletter_exchange, + const bool persistent_); private: Context global_context; @@ -92,6 +93,7 @@ private: const bool use_transactional_channel; const String queue_base; const String deadletter_exchange; + const bool persistent; Poco::Logger * log; std::pair parsed_address; @@ -106,7 +108,7 @@ private: std::vector buffers; /// available buffers for RabbitMQ consumers String local_exchange, bridge_exchange, consumer_exchange; - std::mutex bridge; + std::once_flag flag; AMQP::Table bind_headers; size_t next_channel_id = 1; /// Must >= 1 because it is used as a binding key, which has to be > 0 bool update_channel_id = false; diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 27e4a7b8a03..990f70e0d64 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -16,13 +16,13 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_CONNECT_RABBITMQ; - extern const int LOGICAL_ERROR; } static const auto QUEUE_SIZE = 50000; static const auto CONNECT_SLEEP = 200; static const auto RETRIES_MAX = 1000; static const auto LOOP_WAIT = 10; +static const auto BATCH = 10000; WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( std::pair & parsed_address, @@ -33,7 +33,8 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( const AMQP::ExchangeType exchange_type_, Poco::Logger * log_, size_t num_queues_, - bool use_transactional_channel_, + const bool use_transactional_channel_, + const bool persistent_, std::optional delimiter, size_t rows_per_message, size_t chunk_size_) @@ -44,6 +45,7 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( , exchange_type(exchange_type_) , num_queues(num_queues_) , use_transactional_channel(use_transactional_channel_) + , persistent(persistent_) , payloads(QUEUE_SIZE * num_queues) , log(log_) , delim(delimiter) @@ -57,10 +59,7 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( event_handler = std::make_unique(loop.get(), log); connection = std::make_unique(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); - /* The reason behind making a separate connection for each concurrent producer is explained here: - * https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086 - publishing from - * different threads (as outputStreams are asynchronous) with the same connection leads to internal library errors. - */ + /// New coonection for each publisher because cannot publish from different threads.(https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) size_t cnt_retries = 0; while (!connection->ready() && ++cnt_retries != RETRIES_MAX) { @@ -74,12 +73,27 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( } producer_channel = std::make_shared(connection.get()); + producer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Prodcuer error: {}", message); + }); - /// If publishing should be wrapped in transactions if (use_transactional_channel) { producer_channel->startTransaction(); } + else + { + producer_channel->confirmSelect() + .onAck([&](uint64_t deliveryTag, bool /* multiple */) + { + if (deliveryTag > last_processed) + last_processed = deliveryTag; + }) + .onNack([&](uint64_t /* deliveryTag */, bool /* multiple */, bool /* requeue */) + { + }); + } writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); writing_task->deactivate(); @@ -99,10 +113,7 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( WriteBufferToRabbitMQProducer::~WriteBufferToRabbitMQProducer() { - stop_loop.store(true); writing_task->deactivate(); - initExchange(); - connection->close(); assert(rows == 0 && chunks.empty()); } @@ -130,6 +141,7 @@ void WriteBufferToRabbitMQProducer::countRow() chunks.clear(); set(nullptr, 0); + ++delivery_tag; payloads.push(payload); } } @@ -139,52 +151,51 @@ void WriteBufferToRabbitMQProducer::writingFunc() { String payload; - while (!stop_loop || !payloads.empty()) + auto returned_callback = [&](const AMQP::Message & message, int16_t /* code */, const std::string & /* description */) { - while (!payloads.empty()) + payloads.push(std::string(message.body(), message.size())); + //LOG_DEBUG(log, "Message returned with code: {}, description: {}. Republishing", code, description); + }; + + while ((!payloads.empty() || wait_all) && connection->usable()) + { + while (!payloads.empty() && producer_channel->usable()) { payloads.pop(payload); + AMQP::Envelope envelope(payload.data(), payload.size()); + current = wait_num ? ++current % wait_num : ++current; + + /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse. + if (persistent) + envelope.setDeliveryMode(2); if (exchange_type == AMQP::ExchangeType::consistent_hash) { - next_queue = next_queue % num_queues + 1; - producer_channel->publish(exchange_name, std::to_string(next_queue), payload); + producer_channel->publish(exchange_name, std::to_string(current), envelope).onReturned(returned_callback); } else if (exchange_type == AMQP::ExchangeType::headers) { - AMQP::Envelope envelope(payload.data(), payload.size()); envelope.setHeaders(key_arguments); - producer_channel->publish(exchange_name, "", envelope, key_arguments); + producer_channel->publish(exchange_name, "", envelope, key_arguments).onReturned(returned_callback); } else { - producer_channel->publish(exchange_name, routing_keys[0], payload); + producer_channel->publish(exchange_name, routing_keys[0], envelope).onReturned(returned_callback); } + + if (current % BATCH == 0) + iterateEventLoop(); } - iterateEventLoop(); - } -} - - -void WriteBufferToRabbitMQProducer::initExchange() -{ - std::atomic exchange_declared = false, exchange_error = false; - - producer_channel->declareExchange(exchange_name, exchange_type, AMQP::durable + AMQP::passive) - .onSuccess([&]() - { - exchange_declared = true; - }) - .onError([&](const char * /* message */) - { - exchange_error = true; - }); - - /// These variables are updated in a separate thread. - while (!exchange_declared && !exchange_error) - { - iterateEventLoop(); + if (wait_num.load() && last_processed.load() >= wait_num.load()) + { + wait_all.store(false); + LOG_DEBUG(log, "All messages are successfully published"); + } + else + { + iterateEventLoop(); + } } } @@ -198,11 +209,13 @@ void WriteBufferToRabbitMQProducer::finilizeProducer() .onSuccess([&]() { answer_received = true; + wait_all.store(false); LOG_TRACE(log, "All messages were successfully published"); }) .onError([&](const char * message1) { answer_received = true; + wait_all.store(false); wait_rollback = true; LOG_TRACE(log, "Publishing not successful: {}", message1); producer_channel->rollbackTransaction() diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 20b133b6930..ebeb21075bf 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -28,7 +28,8 @@ public: const AMQP::ExchangeType exchange_type_, Poco::Logger * log_, size_t num_queues_, - bool use_transactional_channel_, + const bool use_transactional_channel_, + const bool persistent_, std::optional delimiter, size_t rows_per_message, size_t chunk_size_ @@ -39,10 +40,10 @@ public: void countRow(); void activateWriting() { writing_task->activateAndSchedule(); } void finilizeProducer(); + void updateMaxWait() { wait_num.store(delivery_tag); } private: void nextImpl() override; - void initExchange(); void iterateEventLoop(); void writingFunc(); @@ -52,10 +53,10 @@ private: AMQP::ExchangeType exchange_type; const size_t num_queues; const bool use_transactional_channel; + const bool persistent; AMQP::Table key_arguments; BackgroundSchedulePool::TaskHolder writing_task; - std::atomic stop_loop = false; std::unique_ptr loop; std::unique_ptr event_handler; @@ -63,7 +64,9 @@ private: ChannelPtr producer_channel; ConcurrentBoundedQueue payloads; - size_t next_queue = 0; + UInt64 delivery_tag = 0, current = 0; + std::atomic wait_all = true; + std::atomic wait_num = 0, last_processed = 0; Poco::Logger * log; const std::optional delim; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 3a2b6cd6be3..abf0a20d18f 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -845,7 +845,6 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): rabbitmq_exchange_name = 'over', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'over', - rabbitmq_num_consumers = 6, rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.rabbitmq_overload (key UInt64, value UInt64) @@ -892,7 +891,6 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): while True: result = instance.query('SELECT count() FROM test.view_overload') time.sleep(1) - print("Result", int(result), "Expected", messages_num * threads_num) if int(result) == messages_num * threads_num: break @@ -1539,7 +1537,7 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): result = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") assert int(result) == 1 - result = instance.query("SELECT key, value, exchange_name, SUBSTRING(consumer_tag, 1, 8), delivery_tag, redelivered FROM test.view") + result = instance.query("SELECT key, value, exchange_name, SUBSTRING(consumer_tag, 1, 8), delivery_tag, redelivered FROM test.view ORDER BY delivery_tag") expected = '''\ 0 0 virtuals_mv amq.ctag 1 0 1 1 virtuals_mv amq.ctag 2 0 @@ -1793,7 +1791,7 @@ def test_rabbitmq_consumer_acknowledgements(rabbitmq_cluster): DROP TABLE IF EXISTS test.rabbitmq_consumer_acks; ''') - collected = int(instance.query('SELECT count() FROM test.view')) + #collected = int(instance.query('SELECT count() FROM test.view')) instance.query(''' CREATE TABLE test.rabbitmq_consumer_acks (key UInt64, value UInt64) @@ -1808,10 +1806,10 @@ def test_rabbitmq_consumer_acknowledgements(rabbitmq_cluster): result1 = instance.query('SELECT count() FROM test.view') time.sleep(1) #print("receiived", result1, "collected", collected) - if int(result1) == messages_num * threads_num: + if int(result1) >= messages_num * threads_num: break - result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") + #result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") instance.query(''' DROP TABLE IF EXISTS test.rabbitmq_consumer_acks; @@ -1819,9 +1817,8 @@ def test_rabbitmq_consumer_acknowledgements(rabbitmq_cluster): DROP TABLE IF EXISTS test.view; ''') - assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) - if collected < result1: - assert int(result2) == 2 + # >= because at-least-once + assert int(result1) >= messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) if __name__ == '__main__': From 92efb847534d5fd088f404153452b46ecc6d7c79 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 25 Jul 2020 15:53:55 +0000 Subject: [PATCH 0021/1911] Update docs --- .../table-engines/integrations/rabbitmq.md | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index 7d09c6f72a5..e870471b4eb 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -44,8 +44,11 @@ Optional parameters: - `rabbitmq_routing_key_list` – A comma-separated list of routing keys. - `rabbitmq_row_delimiter` – Delimiter character, which ends the message. - `rabbitmq_num_consumers` – The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. -- `rabbitmq_num_queues` – The number of queues per consumer. Default: `1`. Specify more queues if the capacity of one queue per consumer is insufficient. Single queue can contain up to 50K messages at the same time. +- `rabbitmq_num_queues` – The number of queues per consumer. Default: `1`. Specify more queues if the capacity of one queue per consumer is insufficient. - `rabbitmq_transactional_channel` – Wrap insert queries in transactions. Default: `0`. +- `rabbitmq_queue_base` - Specify a base name for queues that will be declared. This settings should be used to be able to restore reading from declared durable queues in case of some failure when not all messages were successfully consumed. Note: it makes sence only if messages are sent with delivery mode 2 (marked 'persistent', durable). To be able to resume consumption from one specific queue in case of failure - set its name in `rabbitmq_queue_base` setting and do not specify `rabbitmq_num_consumers` and `rabbitmq_num_queues` (defaults to 1). To be able to resume consumption from all queues, which were declared for a specific table - just specify the same settings: `rabbitmq_queue_base`, `rabbitmq_num_consumers`, `rabbitmq_num_queues`. +- `rabbitmq_deadletter_exchange` - Specify name for a [dead letter exchange](https://www.rabbitmq.com/dlx.html). You can create another table with this exchange name and collect messages in cases when they are republished to dead letter exchange. By default dead letter exchange is not specified. +- `persistent` - If set to 1 (true), in insert query delivery mode will be set to 2 (marks messages as 'persistent'). Default: `0`. Required configuration: @@ -92,13 +95,13 @@ Exchange type options: - `headers` - Routing is based on `key=value` matches with a setting `x-match=all` or `x-match=any`. Example table key list: `x-match=all,format=logs,type=report,year=2020`. - `consistent-hash` - Data is evenly distributed between all bound tables (where exchange name is the same). Note that this exchange type must be enabled with RabbitMQ plugin: `rabbitmq-plugins enable rabbitmq_consistent_hash_exchange`. -If exchange type is not specified, then default is `fanout` and routing keys for data publishing must be randomized in range `[1, num_consumers]` for every message/batch (or in range `[1, num_consumers * num_queues]` if `rabbitmq_num_queues` is set). This table configuration works quicker then any other, especially when `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` parameters are set. - -If `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` parameters are specified along with `rabbitmq_exchange_type`, then: +If `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` settings are specified along with `rabbitmq_exchange_type`, then: - `rabbitmq-consistent-hash-exchange` plugin must be enabled. - `message_id` property of the published messages must be specified (unique for each message/batch). +Do not use the same table for inserts and materialized views. + Example: ``` sql @@ -120,3 +123,10 @@ Example: SELECT key, value FROM daily ORDER BY key; ``` + +## Virtual Columns {#virtual-columns} + +- `_exchange_name` - RabbitMQ exchange name. +- `_consumer_tag` - ConsumerTag of the consumer that received the message. +- `_delivery_tag` - DeliveryTag if the message. Scoped per consumer. +- `_redelivered` - Redelivered flag of the message. From 0ee54c8a4a25c5f7fe16a0a4bb22e9236637089b Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 28 Jul 2020 08:22:45 +0000 Subject: [PATCH 0022/1911] Fix build, async acks -> sync acks, fix tests --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 33 +++-- .../RabbitMQ/RabbitMQBlockInputStream.h | 1 - .../ReadBufferFromRabbitMQConsumer.cpp | 28 ++-- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 9 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 3 - .../WriteBufferToRabbitMQProducer.cpp | 1 + .../integration/test_storage_rabbitmq/test.py | 133 +++++++----------- 7 files changed, 83 insertions(+), 125 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index 0c70acaf1e3..1a20699d23a 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -124,23 +124,26 @@ Block RabbitMQBlockInputStream::readImpl() auto new_rows = read_rabbitmq_message(); - auto exchange_name = storage.getExchange(); - auto consumer_tag = buffer->getConsumerTag(); - auto delivery_tag = buffer->getDeliveryTag(); - auto redelivered = buffer->getRedelivered(); - - for (size_t i = 0; i < new_rows; ++i) + if (new_rows) { - virtual_columns[0]->insert(exchange_name); - virtual_columns[1]->insert(consumer_tag); - virtual_columns[2]->insert(delivery_tag); - virtual_columns[3]->insert(redelivered); + auto exchange_name = storage.getExchange(); + auto consumer_tag = buffer->getConsumerTag(); + auto delivery_tag = buffer->getDeliveryTag(); + auto redelivered = buffer->getRedelivered(); + + buffer->updateNextDeliveryTag(delivery_tag); + + for (size_t i = 0; i < new_rows; ++i) + { + virtual_columns[0]->insert(exchange_name); + virtual_columns[1]->insert(consumer_tag); + virtual_columns[2]->insert(delivery_tag); + virtual_columns[3]->insert(redelivered); + } + + total_rows = total_rows + new_rows; } - if (delivery_tag > last_inserted_delivery_tag) - last_inserted_delivery_tag = delivery_tag; - - total_rows = total_rows + new_rows; buffer->allowNext(); if (!new_rows || !checkTimeLimit()) @@ -167,7 +170,7 @@ void RabbitMQBlockInputStream::readSuffixImpl() if (!buffer) return; - buffer->ackMessages(last_inserted_delivery_tag); + buffer->ackMessages(); } } diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h index 09cda6ff94f..f4405ce44df 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.h @@ -39,7 +39,6 @@ private: const Block virtual_header; ConsumerBufferPtr buffer; - UInt64 last_inserted_delivery_tag; }; } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index b1e63005126..27bb7c12d3d 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -14,7 +14,7 @@ namespace DB { -static const auto QUEUE_SIZE = 50000; /// Equals capacity of a single rabbitmq queue +static const auto QUEUE_SIZE = 50000; ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ChannelPtr consumer_channel_, @@ -57,10 +57,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() { - if (ack.load() && max_tag && consumer_channel) - consumer_channel->ack(max_tag, AMQP::multiple); consumer_channel->close(); - received.clear(); BufferBase::set(nullptr, 0, 0); } @@ -68,7 +65,7 @@ ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) { - bool bindings_created = false, bindings_error = false; + std::atomic bindings_created = false, bindings_error = false; auto success_callback = [&](const std::string & queue_name_, int msgcount, int /* consumercount */) { @@ -220,13 +217,6 @@ void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name) message_received += row_delimiter; received.push({deliveryTag, message_received, redelivered}); - - std::lock_guard lock(wait_ack); - if (ack.exchange(false) && prev_tag && prev_tag <= max_tag && consumer_channel) - { - consumer_channel->ack(prev_tag, AMQP::multiple); /// Will ack all up to last tag staring from last acked. - LOG_TRACE(log, "Consumer {} acknowledged messages with deliveryTags up to {}", consumer_tag, prev_tag); - } } }) .onError([&](const char * message) @@ -239,7 +229,7 @@ void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name) void ReadBufferFromRabbitMQConsumer::checkSubscription() { - if (count_subscribed == num_queues) + if (count_subscribed == num_queues || !consumer_channel->usable()) return; wait_subscribed = num_queues; @@ -264,13 +254,14 @@ void ReadBufferFromRabbitMQConsumer::checkSubscription() } -void ReadBufferFromRabbitMQConsumer::ackMessages(UInt64 last_inserted_delivery_tag) +void ReadBufferFromRabbitMQConsumer::ackMessages() { - if (last_inserted_delivery_tag > prev_tag) + UInt64 delivery_tag = last_inserted_delivery_tag; + if (delivery_tag && delivery_tag > prev_tag) { - std::lock_guard lock(wait_ack); /// See onReceived() callback. - prev_tag = last_inserted_delivery_tag; - ack.store(true); + prev_tag = delivery_tag; + consumer_channel->ack(prev_tag, AMQP::multiple); /// Will ack all up to last tag staring from last acked. + LOG_TRACE(log, "Consumer {} acknowledged messages with deliveryTags up to {}", consumer_tag, prev_tag); } } @@ -291,7 +282,6 @@ bool ReadBufferFromRabbitMQConsumer::nextImpl() auto * new_position = const_cast(current.message.data()); BufferBase::set(new_position, current.message.size(), 0); allowed = false; - max_tag = current.delivery_tag; return true; } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 6d2deb0be03..4854858c9b9 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -50,7 +50,8 @@ public: void allowNext() { allowed = true; } // Allow to read next message. void checkSubscription(); - void ackMessages(UInt64 last_inserted_delivery_tag); + void updateNextDeliveryTag(UInt64 delivery_tag) { last_inserted_delivery_tag = delivery_tag; } + void ackMessages(); auto getConsumerTag() const { return consumer_tag; } auto getDeliveryTag() const { return current.delivery_tag; } @@ -80,18 +81,16 @@ private: String consumer_tag; ConcurrentBoundedQueue received; - UInt64 prev_tag = 0, max_tag = 0; + UInt64 last_inserted_delivery_tag = 0, prev_tag = 0; MessageData current; std::vector queues; std::unordered_map subscribed_queue; - std::atomic ack = false; - std::mutex wait_ack; bool nextImpl() override; void initQueueBindings(const size_t queue_id); void subscribe(const String & queue_name); void iterateEventLoop(); - }; + } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index daa17719654..6a842a69550 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -284,9 +284,6 @@ void StorageRabbitMQ::unbindExchange() { std::call_once(flag, [&]() { - if (exchange_removed.load()) - return; - setup_channel->removeExchange(bridge_exchange) .onSuccess([&]() { diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 990f70e0d64..1b48232aa52 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -150,6 +150,7 @@ void WriteBufferToRabbitMQProducer::countRow() void WriteBufferToRabbitMQProducer::writingFunc() { String payload; + current = 0; auto returned_callback = [&](const AMQP::Message & message, int16_t /* code */, const std::string & /* description */) { diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index abf0a20d18f..bc4585fb6f2 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -120,7 +120,6 @@ def test_rabbitmq_select_from_new_syntax_table(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_routing_key_list = 'new', rabbitmq_exchange_name = 'new', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -136,13 +135,13 @@ def test_rabbitmq_select_from_new_syntax_table(rabbitmq_cluster): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='new', routing_key='new', body=message) + channel.basic_publish(exchange='new', routing_key='', body=message) messages = [] for i in range(25, 50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='new', routing_key='new', body=message) + channel.basic_publish(exchange='new', routing_key='', body=message) connection.close() @@ -191,7 +190,6 @@ def test_rabbitmq_select_empty(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_routing_key_list = 'empty', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; ''') @@ -205,8 +203,7 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_routing_key_list = 'json', - rabbitmq_exchange_name = 'delim1', + rabbitmq_exchange_name = 'json', rabbitmq_format = 'JSONEachRow' ''') @@ -221,14 +218,14 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): all_messages = [messages] for message in all_messages: - channel.basic_publish(exchange='delim1', routing_key='json', body=message) + channel.basic_publish(exchange='json', routing_key='', body=message) messages = '' for i in range(25, 50): messages += json.dumps({'key': i, 'value': i}) + '\n' all_messages = [messages] for message in all_messages: - channel.basic_publish(exchange='delim1', routing_key='json', body=message) + channel.basic_publish(exchange='json', routing_key='', body=message) result = '' while True: @@ -246,8 +243,7 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_routing_key_list = 'csv', - rabbitmq_exchange_name = 'delim2', + rabbitmq_exchange_name = 'csv', rabbitmq_format = 'CSV', rabbitmq_row_delimiter = '\\n'; ''') @@ -262,7 +258,7 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): messages.append('{i}, {i}'.format(i=i)) for message in messages: - channel.basic_publish(exchange='delim2', routing_key='csv', body=message) + channel.basic_publish(exchange='csv', routing_key='', body=message) result = '' while True: @@ -281,8 +277,7 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_routing_key_list = 'tsv', - rabbitmq_exchange_name = 'delim3', + rabbitmq_exchange_name = 'tsv', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; ''') @@ -297,7 +292,7 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): messages.append('{i}\t{i}'.format(i=i)) for message in messages: - channel.basic_publish(exchange='delim3', routing_key='tsv', body=message) + channel.basic_publish(exchange='tsv', routing_key='', body=message) result = '' while True: @@ -318,7 +313,6 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'mv', - rabbitmq_routing_key_list = 'mv', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64) @@ -337,7 +331,7 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='mv', routing_key='mv', body=message) + channel.basic_publish(exchange='mv', routing_key='', body=message) while True: result = instance.query('SELECT * FROM test.view') @@ -362,7 +356,6 @@ def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'mvsq', - rabbitmq_routing_key_list = 'mvsq', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64) @@ -381,7 +374,7 @@ def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='mvsq', routing_key='mvsq', body=message) + channel.basic_publish(exchange='mvsq', routing_key='', body=message) while True: result = instance.query('SELECT * FROM test.view') @@ -408,7 +401,6 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'mmv', - rabbitmq_routing_key_list = 'mmv', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view1 (key UInt64, value UInt64) @@ -432,7 +424,7 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) for message in messages: - channel.basic_publish(exchange='mmv', routing_key='mmv', body=message) + channel.basic_publish(exchange='mmv', routing_key='', body=message) while True: result1 = instance.query('SELECT * FROM test.view1') @@ -470,7 +462,6 @@ def test_rabbitmq_big_message(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'big', - rabbitmq_routing_key_list = 'big', rabbitmq_format = 'JSONEachRow'; CREATE TABLE test.view (key UInt64, value String) ENGINE = MergeTree @@ -480,7 +471,7 @@ def test_rabbitmq_big_message(rabbitmq_cluster): ''') for message in messages: - channel.basic_publish(exchange='big', routing_key='big', body=message) + channel.basic_publish(exchange='big', routing_key='', body=message) while True: result = instance.query('SELECT count() FROM test.view') @@ -580,7 +571,8 @@ def test_rabbitmq_read_only_combo(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'combo', - rabbitmq_num_consumers = 4, + rabbitmq_num_consumers = 2, + rabbitmq_num_queues = 2, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; ''') @@ -614,12 +606,11 @@ def test_rabbitmq_read_only_combo(rabbitmq_cluster): for _ in range(messages_num): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 - key = str(randrange(1, NUM_CONSUMERS)) current = 0 for message in messages: current += 1 mes_id = str(current) - channel.basic_publish(exchange='combo', routing_key=key, + channel.basic_publish(exchange='combo', routing_key='', properties=pika.BasicProperties(message_id=mes_id), body=message) connection.close() @@ -911,8 +902,7 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): def test_rabbitmq_direct_exchange(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, - _consumed_by LowCardinality(String)) + CREATE TABLE test.destination(key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3; @@ -927,14 +917,15 @@ def test_rabbitmq_direct_exchange(rabbitmq_cluster): CREATE TABLE test.direct_exchange_{0} (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_num_consumers = 5, + rabbitmq_num_consumers = 2, + rabbitmq_num_queues = 2, rabbitmq_exchange_name = 'direct_exchange_testing', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'direct_{0}', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.direct_exchange_{0}_mv TO test.destination AS - SELECT key, value, '{0}' as _consumed_by FROM test.direct_exchange_{0}; + SELECT key, value FROM test.direct_exchange_{0}; '''.format(consumer_id)) i = [0] @@ -985,8 +976,7 @@ def test_rabbitmq_direct_exchange(rabbitmq_cluster): def test_rabbitmq_fanout_exchange(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, - _consumed_by LowCardinality(String)) + CREATE TABLE test.destination(key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; ''') @@ -1000,14 +990,15 @@ def test_rabbitmq_fanout_exchange(rabbitmq_cluster): CREATE TABLE test.fanout_exchange_{0} (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_num_consumers = 5, + rabbitmq_num_consumers = 2, + rabbitmq_num_queues = 2, rabbitmq_routing_key_list = 'key_{0}', rabbitmq_exchange_name = 'fanout_exchange_testing', rabbitmq_exchange_type = 'fanout', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.fanout_exchange_{0}_mv TO test.destination AS - SELECT key, value, '{0}' as _consumed_by FROM test.fanout_exchange_{0}; + SELECT key, value FROM test.fanout_exchange_{0}; '''.format(consumer_id)) i = [0] @@ -1055,8 +1046,7 @@ def test_rabbitmq_fanout_exchange(rabbitmq_cluster): def test_rabbitmq_topic_exchange(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, - _consumed_by LowCardinality(String)) + CREATE TABLE test.destination(key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; ''') @@ -1070,14 +1060,15 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): CREATE TABLE test.topic_exchange_{0} (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_num_consumers = 5, + rabbitmq_num_consumers = 2, + rabbitmq_num_queues = 2, rabbitmq_exchange_name = 'topic_exchange_testing', rabbitmq_exchange_type = 'topic', rabbitmq_routing_key_list = '*.{0}', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.topic_exchange_{0}_mv TO test.destination AS - SELECT key, value, '{0}' as _consumed_by FROM test.topic_exchange_{0}; + SELECT key, value FROM test.topic_exchange_{0}; '''.format(consumer_id)) for consumer_id in range(num_tables): @@ -1088,14 +1079,15 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): CREATE TABLE test.topic_exchange_{0} (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_num_consumers = 4, + rabbitmq_num_consumers = 2, + rabbitmq_num_queues = 2, rabbitmq_exchange_name = 'topic_exchange_testing', rabbitmq_exchange_type = 'topic', rabbitmq_routing_key_list = '*.logs', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.topic_exchange_{0}_mv TO test.destination AS - SELECT key, value, '{0}' as _consumed_by FROM test.topic_exchange_{0}; + SELECT key, value FROM test.topic_exchange_{0}; '''.format(num_tables + consumer_id)) i = [0] @@ -1166,7 +1158,8 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): CREATE TABLE test.{0} (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_num_consumers = 10, + rabbitmq_num_consumers = 4, + rabbitmq_num_queues = 2, rabbitmq_exchange_type = 'consistent_hash', rabbitmq_exchange_name = 'hash_exchange_testing', rabbitmq_format = 'JSONEachRow', @@ -1229,7 +1222,7 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): thread.join() assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) - assert int(result2) >= 30 + assert int(result2) == 4 * num_tables @@ -1237,34 +1230,15 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): def test_rabbitmq_multiple_bindings(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, - _consumed_by LowCardinality(String)) + CREATE TABLE test.destination(key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; ''') instance.query(''' - DROP TABLE IF EXISTS test.bindings_1; - DROP TABLE IF EXISTS test.bindings_1_mv; - CREATE TABLE test.bindings_1 (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_num_consumers = 5, - rabbitmq_num_queues = 2, - rabbitmq_exchange_name = 'multiple_bindings_testing', - rabbitmq_exchange_type = 'direct', - rabbitmq_routing_key_list = 'key1,key2,key3,key4,key5', - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - CREATE MATERIALIZED VIEW test.bindings_1_mv TO test.destination AS - SELECT * FROM test.bindings_1; - ''') - - # in case num_consumers and num_queues are not set - multiple bindings are implemented differently, so test them too - instance.query(''' - DROP TABLE IF EXISTS test.bindings_2; - DROP TABLE IF EXISTS test.bindings_2_mv; - CREATE TABLE test.bindings_2 (key UInt64, value UInt64) + DROP TABLE IF EXISTS test.bindings; + DROP TABLE IF EXISTS test.bindings_mv; + CREATE TABLE test.bindings (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'multiple_bindings_testing', @@ -1272,8 +1246,8 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): rabbitmq_routing_key_list = 'key1,key2,key3,key4,key5', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; - CREATE MATERIALIZED VIEW test.bindings_2_mv TO test.destination AS - SELECT * FROM test.bindings_2; + CREATE MATERIALIZED VIEW test.bindings_mv TO test.destination AS + SELECT * FROM test.bindings; ''') i = [0] @@ -1295,12 +1269,8 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): keys = ['key1', 'key2', 'key3', 'key4', 'key5'] for key in keys: - current = 0 for message in messages: - current += 1 - mes_id = str(current) - channel.basic_publish(exchange='multiple_bindings_testing', routing_key=key, - properties=pika.BasicProperties(message_id=mes_id), body=message) + channel.basic_publish(exchange='multiple_bindings_testing', routing_key=key, body=message) connection.close() @@ -1316,32 +1286,31 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): while True: result = instance.query('SELECT count() FROM test.destination') time.sleep(1) - if int(result) == messages_num * threads_num * 5 * 2: + if int(result) == messages_num * threads_num * 5: break for thread in threads: thread.join() instance.query(''' - DROP TABLE IF EXISTS test.bindings_1; - DROP TABLE IF EXISTS test.bindings_2; + DROP TABLE IF EXISTS test.bindings; + DROP TABLE IF EXISTS test.bindings_mv; DROP TABLE IF EXISTS test.destination; ''') - assert int(result) == messages_num * threads_num * 5 * 2, 'ClickHouse lost some messages: {}'.format(result) + assert int(result) == messages_num * threads_num * 5, 'ClickHouse lost some messages: {}'.format(result) @pytest.mark.timeout(420) def test_rabbitmq_headers_exchange(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, - _consumed_by LowCardinality(String)) + CREATE TABLE test.destination(key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; ''') - num_tables_to_receive = 3 + num_tables_to_receive = 2 for consumer_id in range(num_tables_to_receive): print("Setting up table {}".format(consumer_id)) instance.query(''' @@ -1350,14 +1319,14 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): CREATE TABLE test.headers_exchange_{0} (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_num_consumers = 4, + rabbitmq_num_consumers = 2, rabbitmq_exchange_name = 'headers_exchange_testing', rabbitmq_exchange_type = 'headers', rabbitmq_routing_key_list = 'x-match=all,format=logs,type=report,year=2020', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.headers_exchange_{0}_mv TO test.destination AS - SELECT key, value, '{0}' as _consumed_by FROM test.headers_exchange_{0}; + SELECT key, value FROM test.headers_exchange_{0}; '''.format(consumer_id)) num_tables_to_ignore = 2 @@ -1375,7 +1344,7 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.headers_exchange_{0}_mv TO test.destination AS - SELECT key, value, '{0}' as _consumed_by FROM test.headers_exchange_{0}; + SELECT key, value FROM test.headers_exchange_{0}; '''.format(consumer_id + num_tables_to_receive)) i = [0] @@ -1683,7 +1652,7 @@ def test_rabbitmq_queue_resume_2(rabbitmq_cluster): connection.close() threads = [] - threads_num = 10 + threads_num = 20 for _ in range(threads_num): threads.append(threading.Thread(target=produce)) for thread in threads: From 469e46a53904a1d765fb94ab393e60f3b82dc225 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 28 Jul 2020 12:41:58 +0000 Subject: [PATCH 0023/1911] Fix build --- .../RabbitMQ/WriteBufferToRabbitMQProducer.cpp | 11 +++++++---- src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h | 2 +- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 1b48232aa52..82cb3f2311d 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -150,7 +150,7 @@ void WriteBufferToRabbitMQProducer::countRow() void WriteBufferToRabbitMQProducer::writingFunc() { String payload; - current = 0; + UInt64 message_id = 0; auto returned_callback = [&](const AMQP::Message & message, int16_t /* code */, const std::string & /* description */) { @@ -164,7 +164,10 @@ void WriteBufferToRabbitMQProducer::writingFunc() { payloads.pop(payload); AMQP::Envelope envelope(payload.data(), payload.size()); - current = wait_num ? ++current % wait_num : ++current; + + ++message_id; + if (wait_num) + message_id %= wait_num; /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse. if (persistent) @@ -172,7 +175,7 @@ void WriteBufferToRabbitMQProducer::writingFunc() if (exchange_type == AMQP::ExchangeType::consistent_hash) { - producer_channel->publish(exchange_name, std::to_string(current), envelope).onReturned(returned_callback); + producer_channel->publish(exchange_name, std::to_string(message_id), envelope).onReturned(returned_callback); } else if (exchange_type == AMQP::ExchangeType::headers) { @@ -184,7 +187,7 @@ void WriteBufferToRabbitMQProducer::writingFunc() producer_channel->publish(exchange_name, routing_keys[0], envelope).onReturned(returned_callback); } - if (current % BATCH == 0) + if (message_id % BATCH == 0) iterateEventLoop(); } diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index ebeb21075bf..30e647af471 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -64,7 +64,7 @@ private: ChannelPtr producer_channel; ConcurrentBoundedQueue payloads; - UInt64 delivery_tag = 0, current = 0; + UInt64 delivery_tag = 0; std::atomic wait_all = true; std::atomic wait_num = 0, last_processed = 0; From 230938d3a3082fbf241c9d873571231a69a5f450 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 11 Jul 2020 15:12:42 +0800 Subject: [PATCH 0024/1911] Refactor joinGet and implement multi-key lookup. --- src/Functions/FunctionJoinGet.cpp | 83 +++++++++---------- src/Functions/FunctionJoinGet.h | 11 +-- src/Interpreters/HashJoin.cpp | 69 ++++++++------- src/Interpreters/HashJoin.h | 10 +-- src/Interpreters/misc.h | 2 +- .../0_stateless/01080_join_get_null.reference | 2 +- .../0_stateless/01080_join_get_null.sql | 12 +-- .../01400_join_get_with_multi_keys.reference | 1 + .../01400_join_get_with_multi_keys.sql | 9 ++ 9 files changed, 104 insertions(+), 95 deletions(-) create mode 100644 tests/queries/0_stateless/01400_join_get_with_multi_keys.reference create mode 100644 tests/queries/0_stateless/01400_join_get_with_multi_keys.sql diff --git a/src/Functions/FunctionJoinGet.cpp b/src/Functions/FunctionJoinGet.cpp index a33b70684a5..1badc689c6a 100644 --- a/src/Functions/FunctionJoinGet.cpp +++ b/src/Functions/FunctionJoinGet.cpp @@ -1,10 +1,10 @@ #include +#include #include #include #include #include -#include #include @@ -16,19 +16,35 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +template +void ExecutableFunctionJoinGet::execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t) +{ + Block keys; + for (size_t i = 2; i < arguments.size(); ++i) + { + auto key = block.getByPosition(arguments[i]); + keys.insert(std::move(key)); + } + block.getByPosition(result) = join->joinGet(keys, result_block); +} + +template +ExecutableFunctionImplPtr FunctionJoinGet::prepare(const Block &, const ColumnNumbers &, size_t) const +{ + return std::make_unique>(join, Block{{return_type->createColumn(), return_type, attr_name}}); +} + static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & context) { - if (arguments.size() != 3) - throw Exception{"Function joinGet takes 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - String join_name; if (const auto * name_col = checkAndGetColumnConst(arguments[0].column.get())) { join_name = name_col->getValue(); } else - throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function joinGet, expected a const string.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception( + "Illegal type " + arguments[0].type->getName() + " of first argument of function joinGet, expected a const string.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); size_t dot = join_name.find('.'); String database_name; @@ -43,10 +59,12 @@ static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & co ++dot; } String table_name = join_name.substr(dot); + if (table_name.empty()) + throw Exception("joinGet does not allow empty table name", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); auto table = DatabaseCatalog::instance().getTable({database_name, table_name}, context); auto storage_join = std::dynamic_pointer_cast(table); if (!storage_join) - throw Exception{"Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception("Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); String attr_name; if (const auto * name_col = checkAndGetColumnConst(arguments[1].column.get())) @@ -54,57 +72,30 @@ static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & co attr_name = name_col->getValue(); } else - throw Exception{"Illegal type " + arguments[1].type->getName() - + " of second argument of function joinGet, expected a const string.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception( + "Illegal type " + arguments[1].type->getName() + " of second argument of function joinGet, expected a const string.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_pair(storage_join, attr_name); } template FunctionBaseImplPtr JoinGetOverloadResolver::build(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const { + if (arguments.size() < 3) + throw Exception( + "Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + + ", should be greater or equal to 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); auto [storage_join, attr_name] = getJoin(arguments, context); auto join = storage_join->getJoin(); - DataTypes data_types(arguments.size()); - + DataTypes data_types(arguments.size() - 2); + for (size_t i = 2; i < arguments.size(); ++i) + data_types[i - 2] = arguments[i].type; + auto return_type = join->joinGetCheckAndGetReturnType(data_types, attr_name, or_null); auto table_lock = storage_join->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); - for (size_t i = 0; i < arguments.size(); ++i) - data_types[i] = arguments[i].type; - - auto return_type = join->joinGetReturnType(attr_name, or_null); return std::make_unique>(table_lock, storage_join, join, attr_name, data_types, return_type); } -template -DataTypePtr JoinGetOverloadResolver::getReturnType(const ColumnsWithTypeAndName & arguments) const -{ - auto [storage_join, attr_name] = getJoin(arguments, context); - auto join = storage_join->getJoin(); - return join->joinGetReturnType(attr_name, or_null); -} - - -template -void ExecutableFunctionJoinGet::execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - auto ctn = block.getByPosition(arguments[2]); - if (isColumnConst(*ctn.column)) - ctn.column = ctn.column->cloneResized(1); - ctn.name = ""; // make sure the key name never collide with the join columns - Block key_block = {ctn}; - join->joinGet(key_block, attr_name, or_null); - auto & result_ctn = key_block.getByPosition(1); - if (isColumnConst(*ctn.column)) - result_ctn.column = ColumnConst::create(result_ctn.column, input_rows_count); - block.getByPosition(result) = result_ctn; -} - -template -ExecutableFunctionImplPtr FunctionJoinGet::prepare(const Block &, const ColumnNumbers &, size_t) const -{ - return std::make_unique>(join, attr_name); -} - void registerFunctionJoinGet(FunctionFactory & factory) { // joinGet diff --git a/src/Functions/FunctionJoinGet.h b/src/Functions/FunctionJoinGet.h index a82da589960..6b3b1202f60 100644 --- a/src/Functions/FunctionJoinGet.h +++ b/src/Functions/FunctionJoinGet.h @@ -13,14 +13,14 @@ template class ExecutableFunctionJoinGet final : public IExecutableFunctionImpl { public: - ExecutableFunctionJoinGet(HashJoinPtr join_, String attr_name_) - : join(std::move(join_)), attr_name(std::move(attr_name_)) {} + ExecutableFunctionJoinGet(HashJoinPtr join_, const Block & result_block_) + : join(std::move(join_)), result_block(result_block_) {} static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet"; bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } bool useDefaultImplementationForLowCardinalityColumns() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; @@ -28,7 +28,7 @@ public: private: HashJoinPtr join; - const String attr_name; + Block result_block; }; template @@ -77,13 +77,14 @@ public: String getName() const override { return name; } FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const override; - DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const override; + DataTypePtr getReturnType(const ColumnsWithTypeAndName &) const override { return {}; } // Not used bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForLowCardinalityColumns() const override { return true; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } private: const Context & context; diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 27294a57675..ffc806b9e88 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -42,6 +42,7 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; extern const int SET_SIZE_LIMIT_EXCEEDED; extern const int TYPE_MISMATCH; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } namespace @@ -1109,27 +1110,34 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) block = block.cloneWithColumns(std::move(dst_columns)); } -static void checkTypeOfKey(const Block & block_left, const Block & block_right) -{ - const auto & [c1, left_type_origin, left_name] = block_left.safeGetByPosition(0); - const auto & [c2, right_type_origin, right_name] = block_right.safeGetByPosition(0); - auto left_type = removeNullable(left_type_origin); - auto right_type = removeNullable(right_type_origin); - if (!left_type->equals(*right_type)) - throw Exception("Type mismatch of columns to joinGet by: " - + left_name + " " + left_type->getName() + " at left, " - + right_name + " " + right_type->getName() + " at right", - ErrorCodes::TYPE_MISMATCH); -} - - -DataTypePtr HashJoin::joinGetReturnType(const String & column_name, bool or_null) const +DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const { std::shared_lock lock(data->rwlock); + size_t num_keys = data_types.size(); + if (right_table_keys.columns() != num_keys) + throw Exception( + "Number of arguments for function joinGet" + toString(or_null ? "OrNull" : "") + + " doesn't match: passed, should be equal to " + toString(num_keys), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (size_t i = 0; i < num_keys; ++i) + { + const auto & left_type_origin = data_types[i]; + const auto & [c2, right_type_origin, right_name] = right_table_keys.safeGetByPosition(i); + auto left_type = removeNullable(left_type_origin); + auto right_type = removeNullable(right_type_origin); + if (!left_type->equals(*right_type)) + throw Exception( + "Type mismatch in joinGet key " + toString(i) + ": found type " + left_type->getName() + ", while the needed type is " + + right_type->getName(), + ErrorCodes::TYPE_MISMATCH); + } + if (!sample_block_with_columns_to_add.has(column_name)) throw Exception("StorageJoin doesn't contain column " + column_name, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + auto elem = sample_block_with_columns_to_add.getByName(column_name); if (or_null) elem.type = makeNullable(elem.type); @@ -1138,34 +1146,33 @@ DataTypePtr HashJoin::joinGetReturnType(const String & column_name, bool or_null template -void HashJoin::joinGetImpl(Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const +ColumnWithTypeAndName HashJoin::joinGetImpl(const Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const { - joinBlockImpl( - block, {block.getByPosition(0).name}, block_with_columns_to_add, maps_); + // Assemble the key block with correct names. + Block keys; + for (size_t i = 0; i < block.columns(); ++i) + { + auto key = block.getByPosition(i); + key.name = key_names_right[i]; + keys.insert(std::move(key)); + } + + joinBlockImpl( + keys, key_names_right, block_with_columns_to_add, maps_); + return keys.getByPosition(keys.columns() - 1); } -// TODO: support composite key // TODO: return multiple columns as named tuple // TODO: return array of values when strictness == ASTTableJoin::Strictness::All -void HashJoin::joinGet(Block & block, const String & column_name, bool or_null) const +ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const { std::shared_lock lock(data->rwlock); - if (key_names_right.size() != 1) - throw Exception("joinGet only supports StorageJoin containing exactly one key", ErrorCodes::UNSUPPORTED_JOIN_KEYS); - - checkTypeOfKey(block, right_table_keys); - - auto elem = sample_block_with_columns_to_add.getByName(column_name); - if (or_null) - elem.type = makeNullable(elem.type); - elem.column = elem.type->createColumn(); - if ((strictness == ASTTableJoin::Strictness::Any || strictness == ASTTableJoin::Strictness::RightAny) && kind == ASTTableJoin::Kind::Left) { - joinGetImpl(block, {elem}, std::get(data->maps)); + return joinGetImpl(block, block_with_columns_to_add, std::get(data->maps)); } else throw Exception("joinGet only supports StorageJoin of type Left Any", ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN); diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 67d83d27a6d..025f41ac28f 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -162,11 +162,11 @@ public: */ void joinBlock(Block & block, ExtraBlockPtr & not_processed) override; - /// Infer the return type for joinGet function - DataTypePtr joinGetReturnType(const String & column_name, bool or_null) const; + /// Check joinGet arguments and infer the return type. + DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const; - /// Used by joinGet function that turns StorageJoin into a dictionary - void joinGet(Block & block, const String & column_name, bool or_null) const; + /// Used by joinGet function that turns StorageJoin into a dictionary. + ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add) const; /** Keep "totals" (separate part of dataset, see WITH TOTALS) to use later. */ @@ -383,7 +383,7 @@ private: void joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const; template - void joinGetImpl(Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const; + ColumnWithTypeAndName joinGetImpl(const Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const; static Type chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes); }; diff --git a/src/Interpreters/misc.h b/src/Interpreters/misc.h index 094dfbbbb81..cae2691ca1f 100644 --- a/src/Interpreters/misc.h +++ b/src/Interpreters/misc.h @@ -28,7 +28,7 @@ inline bool functionIsLikeOperator(const std::string & name) inline bool functionIsJoinGet(const std::string & name) { - return name == "joinGet" || startsWith(name, "dictGet"); + return startsWith(name, "joinGet"); } inline bool functionIsDictGet(const std::string & name) diff --git a/tests/queries/0_stateless/01080_join_get_null.reference b/tests/queries/0_stateless/01080_join_get_null.reference index bfde072a796..0cfbf08886f 100644 --- a/tests/queries/0_stateless/01080_join_get_null.reference +++ b/tests/queries/0_stateless/01080_join_get_null.reference @@ -1 +1 @@ -2 2 +2 diff --git a/tests/queries/0_stateless/01080_join_get_null.sql b/tests/queries/0_stateless/01080_join_get_null.sql index 71e7ddf8e75..9f782452d34 100644 --- a/tests/queries/0_stateless/01080_join_get_null.sql +++ b/tests/queries/0_stateless/01080_join_get_null.sql @@ -1,12 +1,12 @@ DROP TABLE IF EXISTS test_joinGet; -DROP TABLE IF EXISTS test_join_joinGet; -CREATE TABLE test_joinGet(id Int32, user_id Nullable(Int32)) Engine = Memory(); -CREATE TABLE test_join_joinGet(user_id Int32, name String) Engine = Join(ANY, LEFT, user_id); +CREATE TABLE test_joinGet(user_id Nullable(Int32), name String) Engine = Join(ANY, LEFT, user_id); -INSERT INTO test_join_joinGet VALUES (2, 'a'), (6, 'b'), (10, 'c'); +INSERT INTO test_joinGet VALUES (2, 'a'), (6, 'b'), (10, 'c'), (null, 'd'); -SELECT 2 id, toNullable(toInt32(2)) user_id WHERE joinGet(test_join_joinGet, 'name', user_id) != ''; +SELECT toNullable(toInt32(2)) user_id WHERE joinGet(test_joinGet, 'name', user_id) != ''; + +-- If the JOIN keys are Nullable fields, the rows where at least one of the keys has the value NULL are not joined. +SELECT cast(null AS Nullable(Int32)) user_id WHERE joinGet(test_joinGet, 'name', user_id) != ''; DROP TABLE test_joinGet; -DROP TABLE test_join_joinGet; 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 new file mode 100644 index 00000000000..49d59571fbf --- /dev/null +++ b/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference @@ -0,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 new file mode 100644 index 00000000000..73068270762 --- /dev/null +++ b/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS test_joinGet; + +CREATE TABLE test_joinGet(a String, b String, c Float64) ENGINE = Join(any, left, a, b); + +INSERT INTO test_joinGet VALUES ('ab', '1', 0.1), ('ab', '2', 0.2), ('cd', '3', 0.3); + +SELECT joinGet(test_joinGet, 'c', 'ab', '1'); + +DROP TABLE test_joinGet; From 763c337be99e7bcd482e442c54e383f47c2f1a32 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 29 Jul 2020 19:45:18 +0000 Subject: [PATCH 0025/1911] Remove redundant, move subscription --- .../ReadBufferFromRabbitMQConsumer.cpp | 172 +++++------------- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 11 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 12 +- 3 files changed, 54 insertions(+), 141 deletions(-) diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 27bb7c12d3d..9f036a8a9b6 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -21,15 +21,12 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ChannelPtr setup_channel_, HandlerPtr event_handler_, const String & exchange_name_, - const AMQP::ExchangeType & exchange_type_, - const Names & routing_keys_, size_t channel_id_, const String & queue_base_, Poco::Logger * log_, char row_delimiter_, bool hash_exchange_, size_t num_queues_, - const String & local_exchange_, const String & deadletter_exchange_, const std::atomic & stopped_) : ReadBuffer(nullptr, 0) @@ -37,8 +34,6 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , setup_channel(setup_channel_) , event_handler(event_handler_) , exchange_name(exchange_name_) - , exchange_type(exchange_type_) - , routing_keys(routing_keys_) , channel_id(channel_id_) , queue_base(queue_base_) , hash_exchange(hash_exchange_) @@ -46,24 +41,24 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , log(log_) , row_delimiter(row_delimiter_) , stopped(stopped_) - , local_exchange(local_exchange_) , deadletter_exchange(deadletter_exchange_) , received(QUEUE_SIZE * num_queues) { for (size_t queue_id = 0; queue_id < num_queues; ++queue_id) - initQueueBindings(queue_id); + bindQueue(queue_id); + + consumer_channel->onReady([&]() { subscribe(); }); } ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() { consumer_channel->close(); - received.clear(); BufferBase::set(nullptr, 0, 0); } -void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) +void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) { std::atomic bindings_created = false, bindings_error = false; @@ -75,87 +70,17 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) if (msgcount) LOG_TRACE(log, "Queue " + queue_name_ + " is non-empty. Non-consumed messaged will also be delivered."); - subscribed_queue[queue_name_] = false; - subscribe(queues.back()); - - if (hash_exchange) + /// Binding key must be a string integer in case of hash exchange (here it is either hash or fanout). + setup_channel->bindQueue(exchange_name, queue_name_, std::to_string(channel_id)) + .onSuccess([&] { - String binding_key; - if (queues.size() == 1) - binding_key = std::to_string(channel_id); - else - binding_key = std::to_string(channel_id + queue_id); - - /* If exchange_type == hash, then bind directly to this client's exchange (because there is no need for a distributor - * exchange as it is already hash-exchange), otherwise hash-exchange is a local distributor exchange. - */ - String current_hash_exchange = exchange_type == AMQP::ExchangeType::consistent_hash ? exchange_name : local_exchange; - - setup_channel->bindQueue(current_hash_exchange, queue_name_, binding_key) - .onSuccess([&] - { - bindings_created = true; - }) - .onError([&](const char * message) - { - bindings_error = true; - LOG_ERROR(log, "Failed to create queue binding. Reason: {}", message); - }); - } - else if (exchange_type == AMQP::ExchangeType::fanout) + bindings_created = true; + }) + .onError([&](const char * message) { - setup_channel->bindQueue(exchange_name, queue_name_, routing_keys[0]) - .onSuccess([&] - { - bindings_created = true; - }) - .onError([&](const char * message) - { - bindings_error = true; - LOG_ERROR(log, "Failed to bind to key. Reason: {}", message); - }); - } - else if (exchange_type == AMQP::ExchangeType::headers) - { - AMQP::Table binding_arguments; - std::vector matching; - - for (const auto & header : routing_keys) - { - boost::split(matching, header, [](char c){ return c == '='; }); - binding_arguments[matching[0]] = matching[1]; - matching.clear(); - } - - setup_channel->bindQueue(exchange_name, queue_name_, routing_keys[0], binding_arguments) - .onSuccess([&] - { - bindings_created = true; - }) - .onError([&](const char * message) - { - bindings_error = true; - LOG_ERROR(log, "Failed to bind queue. Reason: {}", message); - }); - } - else - { - /// Means there is only one queue with one consumer - no even distribution needed - no hash-exchange. - for (const auto & routing_key : routing_keys) - { - /// Binding directly to exchange, specified by the client. - setup_channel->bindQueue(exchange_name, queue_name_, routing_key) - .onSuccess([&] - { - bindings_created = true; - }) - .onError([&](const char * message) - { - bindings_error = true; - LOG_ERROR(log, "Failed to bind queue. Reason: {}", message); - }); - } - } + bindings_error = true; + LOG_ERROR(log, "Failed to create queue binding. Reason: {}", message); + }); }; auto error_callback([&](const char * message) @@ -187,43 +112,42 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id) } -void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name) +void ReadBufferFromRabbitMQConsumer::subscribe() { - if (subscribed_queue[queue_name]) - return; - - consumer_channel->consume(queue_name) - .onSuccess([&](const std::string & consumer) + count_subscribed = 0; + for (const auto & queue_name : queues) { - subscribed_queue[queue_name] = true; - ++count_subscribed; - LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); - - consumer_error = false; - consumer_tag = consumer; - - consumer_channel->onError([&](const char * message) + consumer_channel->consume(queue_name) + .onSuccess([&](const std::string & consumer) { - LOG_ERROR(log, "Consumer {} error: {}", consumer_tag, message); + ++count_subscribed; + LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); + + consumer_error = false; + consumer_tag = consumer; + + consumer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Consumer {} error: {}", consumer_tag, message); + }); + }) + .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) + { + if (message.bodySize()) + { + String message_received = std::string(message.body(), message.body() + message.bodySize()); + if (row_delimiter != '\0') + message_received += row_delimiter; + + received.push({delivery_tag, message_received, redelivered}); + } + }) + .onError([&](const char * message) + { + consumer_error = true; + LOG_ERROR(log, "Consumer {} failed. Reason: {}", channel_id, message); }); - }) - .onReceived([&](const AMQP::Message & message, uint64_t deliveryTag, bool redelivered) - { - size_t message_size = message.bodySize(); - if (message_size && message.body() != nullptr) - { - String message_received = std::string(message.body(), message.body() + message_size); - if (row_delimiter != '\0') - message_received += row_delimiter; - - received.push({deliveryTag, message_received, redelivered}); - } - }) - .onError([&](const char * message) - { - consumer_error = true; - LOG_ERROR(log, "Consumer {} failed. Reason: {}", channel_id, message); - }); + } } @@ -246,11 +170,7 @@ void ReadBufferFromRabbitMQConsumer::checkSubscription() if (count_subscribed == num_queues) return; - /// A case that should never normally happen. - for (auto & queue : queues) - { - subscribe(queue); - } + subscribe(); } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 4854858c9b9..6448389aea5 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -27,15 +27,12 @@ public: ChannelPtr setup_channel_, HandlerPtr event_handler_, const String & exchange_name_, - const AMQP::ExchangeType & exchange_type_, - const Names & routing_keys_, size_t channel_id_, const String & queue_base_, Poco::Logger * log_, char row_delimiter_, bool hash_exchange_, size_t num_queues_, - const String & local_exchange_, const String & deadletter_exchange_, const std::atomic & stopped_); @@ -63,8 +60,6 @@ private: HandlerPtr event_handler; const String exchange_name; - const AMQP::ExchangeType exchange_type; - const Names routing_keys; const size_t channel_id; const String queue_base; const bool hash_exchange; @@ -75,7 +70,7 @@ private: bool allowed = true; const std::atomic & stopped; - const String local_exchange, deadletter_exchange; + const String deadletter_exchange; std::atomic consumer_error = false; std::atomic count_subscribed = 0, wait_subscribed; @@ -88,8 +83,8 @@ private: bool nextImpl() override; - void initQueueBindings(const size_t queue_id); - void subscribe(const String & queue_name); + void bindQueue(size_t queue_id); + void subscribe(); void iterateEventLoop(); }; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 6a842a69550..f31cf3f4f72 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -126,7 +126,7 @@ StorageRabbitMQ::StorageRabbitMQ( if (exchange_type_ != ExchangeType::DEFAULT) { - if (exchange_type_ == ExchangeType::FANOUT) exchange_type = AMQP::ExchangeType::fanout; + if (exchange_type_ == ExchangeType::FANOUT) exchange_type = AMQP::ExchangeType::fanout; else if (exchange_type_ == ExchangeType::DIRECT) exchange_type = AMQP::ExchangeType::direct; else if (exchange_type_ == ExchangeType::TOPIC) exchange_type = AMQP::ExchangeType::topic; else if (exchange_type_ == ExchangeType::HASH) exchange_type = AMQP::ExchangeType::consistent_hash; @@ -140,12 +140,11 @@ StorageRabbitMQ::StorageRabbitMQ( if (exchange_type == AMQP::ExchangeType::headers) { - std::vector matching; for (const auto & header : routing_keys) { + std::vector matching; boost::split(matching, header, [](char c){ return c == '='; }); bind_headers[matching[0]] = matching[1]; - matching.clear(); } } @@ -192,7 +191,7 @@ void StorageRabbitMQ::initExchange() + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); }); - /// Bridge exchange is needed to easily disconnect consumer queues. + /// Bridge exchange is needed to easily disconnect consumer queues. Also simplifies queue bindings a lot. setup_channel->declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable + AMQP::autodelete) .onError([&](const char * message) { @@ -230,7 +229,6 @@ void StorageRabbitMQ::bindExchange() std::atomic binding_created = false; size_t bound_keys = 0; - /// Bridge exchange connects client's exchange with consumers' queues. if (exchange_type == AMQP::ExchangeType::headers) { setup_channel->bindExchange(exchange_name, bridge_exchange, routing_keys[0], bind_headers) @@ -434,9 +432,9 @@ ConsumerBufferPtr StorageRabbitMQ::createReadBuffer() ChannelPtr consumer_channel = std::make_shared(connection.get()); return std::make_shared( - consumer_channel, setup_channel, event_handler, consumer_exchange, exchange_type, routing_keys, + consumer_channel, setup_channel, event_handler, consumer_exchange, next_channel_id, queue_base, log, row_delimiter, hash_exchange, num_queues, - local_exchange, deadletter_exchange, stream_cancelled); + deadletter_exchange, stream_cancelled); } From 5a934c079e691d4231b08a1a96204a6ebd5d85d2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 31 Jul 2020 04:59:56 +0000 Subject: [PATCH 0026/1911] Add connection restore in insert, better confirms --- .../WriteBufferToRabbitMQProducer.cpp | 127 +++++++++++++----- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 14 +- 2 files changed, 104 insertions(+), 37 deletions(-) diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 82cb3f2311d..d74e94d74d2 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -25,7 +25,7 @@ static const auto LOOP_WAIT = 10; static const auto BATCH = 10000; WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( - std::pair & parsed_address, + std::pair & parsed_address_, Context & global_context, const std::pair & login_password_, const Names & routing_keys_, @@ -39,6 +39,7 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( size_t rows_per_message, size_t chunk_size_) : WriteBuffer(nullptr, 0) + , parsed_address(parsed_address_) , login_password(login_password_) , routing_keys(routing_keys_) , exchange_name(exchange_name_) @@ -55,11 +56,45 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( loop = std::make_unique(); uv_loop_init(loop.get()); - event_handler = std::make_unique(loop.get(), log); + + /// New coonection for each publisher because cannot publish from different threads with the same connection.(https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) + setupConnection(0); + setupChannel(0); + + writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); + writing_task->deactivate(); + + if (exchange_type == AMQP::ExchangeType::headers) + { + for (const auto & header : routing_keys) + { + std::vector matching; + boost::split(matching, header, [](char c){ return c == '='; }); + key_arguments[matching[0]] = matching[1]; + } + } +} + + +WriteBufferToRabbitMQProducer::~WriteBufferToRabbitMQProducer() +{ + writing_task->deactivate(); + connection->close(); + assert(rows == 0 && chunks.empty()); +} + + +void WriteBufferToRabbitMQProducer::setupConnection(bool remove_prev_connection) +{ + if (remove_prev_connection && connection) + { + connection->close(); + connection.release(); + } + connection = std::make_unique(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); - /// New coonection for each publisher because cannot publish from different threads.(https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) size_t cnt_retries = 0; while (!connection->ready() && ++cnt_retries != RETRIES_MAX) { @@ -71,8 +106,18 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( { throw Exception("Cannot set up connection for producer", ErrorCodes::CANNOT_CONNECT_RABBITMQ); } +} - producer_channel = std::make_shared(connection.get()); + +void WriteBufferToRabbitMQProducer::setupChannel(bool remove_prev_channel) +{ + if (remove_prev_channel && producer_channel) + { + producer_channel->close(); + producer_channel.release(); + } + + producer_channel = std::make_unique(connection.get()); producer_channel->onError([&](const char * message) { LOG_ERROR(log, "Prodcuer error: {}", message); @@ -84,38 +129,38 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( } else { + /// Same as here https://www.rabbitmq.com/blog/2011/02/10/introducing-publisher-confirms/ + remove_confirmed_tag = [&](uint64_t received_delivery_tag, bool multiple) + { + std::lock_guard lock(mutex); + auto found_tag_pos = delivery_tags_record.find(received_delivery_tag); + if (found_tag_pos != delivery_tags_record.end()) + { + if (multiple) + { + ++found_tag_pos; + delivery_tags_record.erase(delivery_tags_record.begin(), found_tag_pos); + } + else + delivery_tags_record.erase(found_tag_pos); + } + }; + + /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, it + * will be requed in returned_callback. If persistent == false, message is confirmed the moment it is enqueued. If fails, it is + * not requeued. First option is two times slower than the second, so default is second and the first is turned on in table setting. + */ producer_channel->confirmSelect() - .onAck([&](uint64_t deliveryTag, bool /* multiple */) + .onAck([&](uint64_t acked_delivery_tag, bool multiple) { - if (deliveryTag > last_processed) - last_processed = deliveryTag; + remove_confirmed_tag(acked_delivery_tag, multiple); }) - .onNack([&](uint64_t /* deliveryTag */, bool /* multiple */, bool /* requeue */) + .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) { + if (!persistent) + remove_confirmed_tag(nacked_delivery_tag, multiple); }); } - - writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); - writing_task->deactivate(); - - if (exchange_type == AMQP::ExchangeType::headers) - { - std::vector matching; - for (const auto & header : routing_keys) - { - boost::split(matching, header, [](char c){ return c == '='; }); - key_arguments[matching[0]] = matching[1]; - matching.clear(); - } - } -} - - -WriteBufferToRabbitMQProducer::~WriteBufferToRabbitMQProducer() -{ - writing_task->deactivate(); - connection->close(); - assert(rows == 0 && chunks.empty()); } @@ -143,6 +188,9 @@ void WriteBufferToRabbitMQProducer::countRow() ++delivery_tag; payloads.push(payload); + + std::lock_guard lock(mutex); + delivery_tags_record.insert(delivery_tags_record.end(), delivery_tag); } } @@ -180,7 +228,7 @@ void WriteBufferToRabbitMQProducer::writingFunc() else if (exchange_type == AMQP::ExchangeType::headers) { envelope.setHeaders(key_arguments); - producer_channel->publish(exchange_name, "", envelope, key_arguments).onReturned(returned_callback); + producer_channel->publish(exchange_name, "", envelope).onReturned(returned_callback); } else { @@ -191,7 +239,7 @@ void WriteBufferToRabbitMQProducer::writingFunc() iterateEventLoop(); } - if (wait_num.load() && last_processed.load() >= wait_num.load()) + if (wait_num.load() && delivery_tags_record.empty()) { wait_all.store(false); LOG_DEBUG(log, "All messages are successfully published"); @@ -200,7 +248,22 @@ void WriteBufferToRabbitMQProducer::writingFunc() { iterateEventLoop(); } + + /// Most channel based errors result in channel closure, which is very likely to trigger connection closure. + if (connection->usable() && connection->ready() && !producer_channel->usable()) + { + LOG_DEBUG(log, "Channel is not usable. Creating a new one"); + setupChannel(1); + } + else if (!connection->usable() || !connection->ready()) + { + LOG_DEBUG(log, "Connection is not usable. Creating a new one"); + setupConnection(1); + setupChannel(1); + } } + + LOG_DEBUG(log, "Delivered messages"); } diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 30e647af471..188bd5676f4 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -14,13 +14,11 @@ namespace DB { -using ChannelPtr = std::shared_ptr; - class WriteBufferToRabbitMQProducer : public WriteBuffer { public: WriteBufferToRabbitMQProducer( - std::pair & parsed_address, + std::pair & parsed_address_, Context & global_context, const std::pair & login_password_, const Names & routing_keys_, @@ -46,7 +44,10 @@ private: void nextImpl() override; void iterateEventLoop(); void writingFunc(); + void setupConnection(bool remove_prev_connection); + void setupChannel(bool remove_prev_channel); + std::pair parsed_address; const std::pair login_password; const Names routing_keys; const String exchange_name; @@ -61,12 +62,15 @@ private: std::unique_ptr loop; std::unique_ptr event_handler; std::unique_ptr connection; - ChannelPtr producer_channel; + std::unique_ptr producer_channel; ConcurrentBoundedQueue payloads; UInt64 delivery_tag = 0; std::atomic wait_all = true; - std::atomic wait_num = 0, last_processed = 0; + std::atomic wait_num = 0; + std::set delivery_tags_record; + std::mutex mutex; + std::function remove_confirmed_tag; Poco::Logger * log; const std::optional delim; From 40504f6a6e9b54bdcdb0c63a5724648bf5bc04f5 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 31 Jul 2020 17:57:00 +0300 Subject: [PATCH 0027/1911] Simpler version of #12999 w/o `pos` changes --- programs/client/Client.cpp | 126 +++++++++++++----- ...06_insert_values_and_expressions.reference | 2 + .../00306_insert_values_and_expressions.sql | 9 ++ 3 files changed, 103 insertions(+), 34 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 797342a1b44..78a6d7fe2d9 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -908,74 +908,127 @@ private: return processMultiQuery(text); } - bool processMultiQuery(const String & text) + bool processMultiQuery(const String & all_queries_text) { const bool test_mode = config().has("testmode"); { /// disable logs if expects errors - TestHint test_hint(test_mode, text); + TestHint test_hint(test_mode, all_queries_text); if (test_hint.clientError() || test_hint.serverError()) processTextAsSingleQuery("SET send_logs_level = 'none'"); } /// Several queries separated by ';'. /// INSERT data is ended by the end of line, not ';'. + /// An exception is VALUES format where we also support semicolon in + /// addition to end of line. - const char * begin = text.data(); - const char * end = begin + text.size(); + const char * this_query_begin = all_queries_text.data(); + const char * all_queries_end = all_queries_text.data() + all_queries_text.size(); - while (begin < end) + while (this_query_begin < all_queries_end) { - const char * pos = begin; - ASTPtr orig_ast = parseQuery(pos, end, true); + // Use the token iterator to skip any whitespace, semicolons and + // comments at the beginning of the query. An example from regression + // tests: + // insert into table t values ('invalid'); -- { serverError 469 } + // select 1 + // Here the test hint comment gets parsed as a part of second query. + // We parse the `INSERT VALUES` up to the semicolon, and the rest + // looks like a two-line query: + // -- { serverError 469 } + // select 1 + // and we expect it to fail with error 469, but this hint is actually + // for the previous query. Test hints should go after the query, so + // we can fix this by skipping leading comments. Token iterator skips + // comments and whitespace by itself, so we only have to check for + // semicolons. + // The code block is to limit visibility of `tokens` because we have + // another such variable further down the code, and get warnings for + // that. + { + Tokens tokens(this_query_begin, all_queries_end); + IParser::Pos token_iterator(tokens, + context.getSettingsRef().max_parser_depth); + while (token_iterator->type == TokenType::Semicolon + && token_iterator.isValid()) + { + ++token_iterator; + } + this_query_begin = token_iterator->begin; + if (this_query_begin >= all_queries_end) + { + break; + } + } - if (!orig_ast) + // Try to parse the query. + const char * this_query_end = this_query_begin; + parsed_query = parseQuery(this_query_end, all_queries_end, true); + + if (!parsed_query) { if (ignore_error) { - Tokens tokens(begin, end); + Tokens tokens(this_query_begin, all_queries_end); IParser::Pos token_iterator(tokens, context.getSettingsRef().max_parser_depth); while (token_iterator->type != TokenType::Semicolon && token_iterator.isValid()) ++token_iterator; - begin = token_iterator->end; + this_query_begin = token_iterator->end; continue; } return true; } - auto * insert = orig_ast->as(); - - if (insert && insert->data) + // INSERT queries may have the inserted data in the query text + // that follow the query itself, e.g. "insert into t format CSV 1;2". + // They need special handling. First of all, here we find where the + // inserted data ends. In multy-query mode, it is delimited by a + // newline. + // The VALUES format needs even more handling -- we also allow the + // data to be delimited by semicolon. This case is handled later by + // the format parser itself. + auto * insert_ast = parsed_query->as(); + if (insert_ast && insert_ast->data) { - pos = find_first_symbols<'\n'>(insert->data, end); - insert->end = pos; + this_query_end = find_first_symbols<'\n'>(insert_ast->data, all_queries_end); + insert_ast->end = this_query_end; + query_to_send = all_queries_text.substr( + this_query_begin - all_queries_text.data(), + insert_ast->data - this_query_begin); + } + else + { + query_to_send = all_queries_text.substr( + this_query_begin - all_queries_text.data(), + this_query_end - this_query_begin); } - String str = text.substr(begin - text.data(), pos - begin); + // full_query is the query + inline INSERT data. + full_query = all_queries_text.substr( + this_query_begin - all_queries_text.data(), + this_query_end - this_query_begin); - begin = pos; - while (isWhitespaceASCII(*begin) || *begin == ';') - ++begin; - - TestHint test_hint(test_mode, str); + // Look for the hint in the text of query + insert data, if any. + // e.g. insert into t format CSV 'a' -- { serverError 123 }. + TestHint test_hint(test_mode, full_query); expected_client_error = test_hint.clientError(); expected_server_error = test_hint.serverError(); try { - auto ast_to_process = orig_ast; - if (insert && insert->data) + processParsedSingleQuery(); + + if (insert_ast && insert_ast->data) { - ast_to_process = nullptr; - processTextAsSingleQuery(str); - } - else - { - parsed_query = ast_to_process; - full_query = str; - query_to_send = str; - processParsedSingleQuery(); + // For VALUES format: use the end of inline data as reported + // by the format parser (it is saved in sendData()). This + // allows us to handle queries like: + // insert into t values (1); select 1 + //, where the inline data is delimited by semicolon and not + // by a newline. + this_query_end = parsed_query->as()->end; } } catch (...) @@ -983,7 +1036,7 @@ private: last_exception_received_from_server = std::make_unique(getCurrentExceptionMessage(true), getCurrentExceptionCode()); actual_client_error = last_exception_received_from_server->code(); if (!ignore_error && (!actual_client_error || actual_client_error != expected_client_error)) - std::cerr << "Error on processing query: " << str << std::endl << last_exception_received_from_server->message(); + std::cerr << "Error on processing query: " << full_query << std::endl << last_exception_received_from_server->message(); received_exception_from_server = true; } @@ -997,6 +1050,8 @@ private: else return false; } + + this_query_begin = this_query_end; } return true; @@ -1407,7 +1462,7 @@ private: void sendData(Block & sample, const ColumnsDescription & columns_description) { /// If INSERT data must be sent. - const auto * parsed_insert_query = parsed_query->as(); + auto * parsed_insert_query = parsed_query->as(); if (!parsed_insert_query) return; @@ -1416,6 +1471,9 @@ private: /// Send data contained in the query. ReadBufferFromMemory data_in(parsed_insert_query->data, parsed_insert_query->end - parsed_insert_query->data); sendDataFrom(data_in, sample, columns_description); + // Remember where the data ended. We use this info later to determine + // where the next query begins. + parsed_insert_query->end = data_in.buffer().begin() + data_in.count(); } else if (!is_interactive) { diff --git a/tests/queries/0_stateless/00306_insert_values_and_expressions.reference b/tests/queries/0_stateless/00306_insert_values_and_expressions.reference index 960773dc489..e80a28accf4 100644 --- a/tests/queries/0_stateless/00306_insert_values_and_expressions.reference +++ b/tests/queries/0_stateless/00306_insert_values_and_expressions.reference @@ -2,3 +2,5 @@ 2 Hello, world 00000000-0000-0000-0000-000000000000 2016-01-02 2016-01-02 03:04:00 [0,1] 3 hello, world! ab41bdd6-5cd4-11e7-907b-a6006ad3dba0 2016-01-03 2016-01-02 03:00:00 [] 4 World ab41bdd6-5cd4-11e7-907b-a6006ad3dba0 2016-01-04 2016-12-11 10:09:08 [3,2,1] +11111 +1 diff --git a/tests/queries/0_stateless/00306_insert_values_and_expressions.sql b/tests/queries/0_stateless/00306_insert_values_and_expressions.sql index a57e9e69fe6..10a1415f287 100644 --- a/tests/queries/0_stateless/00306_insert_values_and_expressions.sql +++ b/tests/queries/0_stateless/00306_insert_values_and_expressions.sql @@ -5,3 +5,12 @@ INSERT INTO insert VALUES (1, 'Hello', 'ab41bdd6-5cd4-11e7-907b-a6006ad3dba0', ' SELECT * FROM insert ORDER BY i; DROP TABLE insert; + +-- Test the case where the VALUES are delimited by semicolon and a query follows +-- w/o newline. With most formats the query in the same line would be ignored or +-- lead to an error, but VALUES are an exception and support semicolon delimiter, +-- in addition to the newline. +create table if not exists t_306 (a int) engine Memory; +insert into t_306 values (1); select 11111; +select * from t_306; +drop table if exists t_306; From c2bed351ae57c6eb69fb04154d7617e4f13a4c8b Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 1 Aug 2020 12:52:00 +0000 Subject: [PATCH 0028/1911] Add consumer connection track and restore --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 3 +- src/Storages/RabbitMQ/RabbitMQHandler.cpp | 18 ++-- src/Storages/RabbitMQ/RabbitMQHandler.h | 4 +- .../ReadBufferFromRabbitMQConsumer.cpp | 76 ++++++++-------- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 7 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 86 +++++++++++++++---- src/Storages/RabbitMQ/StorageRabbitMQ.h | 6 +- 7 files changed, 131 insertions(+), 69 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index 1a20699d23a..589f5b39d2e 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -52,7 +52,8 @@ void RabbitMQBlockInputStream::readPrefixImpl() if (!buffer || finished) return; - buffer->checkSubscription(); + if (!buffer->channelUsable() && (storage.connectionRunning() || storage.restoreConnection())) + buffer->restoreChannel(storage.getChannel()); } diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.cpp b/src/Storages/RabbitMQ/RabbitMQHandler.cpp index 5d17ff23b64..ecaa109c184 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.cpp +++ b/src/Storages/RabbitMQ/RabbitMQHandler.cpp @@ -5,11 +5,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_CONNECT_RABBITMQ; -} - /* The object of this class is shared between concurrent consumers (who share the same connection == share the same * event loop and handler). */ @@ -20,19 +15,26 @@ RabbitMQHandler::RabbitMQHandler(uv_loop_t * loop_, Poco::Logger * log_) : { } +///Method that is called when the connection ends up in an error state. void RabbitMQHandler::onError(AMQP::TcpConnection * connection, const char * message) { + connection_running.store(false); LOG_ERROR(log, "Library error report: {}", message); - if (!connection->usable() || !connection->ready()) - throw Exception("Connection error", ErrorCodes::CANNOT_CONNECT_RABBITMQ); + if (connection) + connection->close(); +} + +void RabbitMQHandler::onReady(AMQP::TcpConnection * /* connection */) +{ + connection_running.store(true); } void RabbitMQHandler::startLoop() { std::lock_guard lock(startup_mutex); /// stop_loop variable is updated in a separate thread - while (!stop_loop.load()) + while (!stop_loop.load() && connection_running.load()) uv_run(loop, UV_RUN_NOWAIT); } diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.h b/src/Storages/RabbitMQ/RabbitMQHandler.h index 5893ace1d2f..2a992f68d27 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.h +++ b/src/Storages/RabbitMQ/RabbitMQHandler.h @@ -17,16 +17,18 @@ class RabbitMQHandler : public AMQP::LibUvHandler public: RabbitMQHandler(uv_loop_t * loop_, Poco::Logger * log_); void onError(AMQP::TcpConnection * connection, const char * message) override; + void onReady(AMQP::TcpConnection * connection) override; void stop() { stop_loop.store(true); } void startLoop(); void iterateLoop(); + bool connectionRunning() { return connection_running.load(); } private: uv_loop_t * loop; Poco::Logger * log; - std::atomic stop_loop = false; + std::atomic stop_loop = false, connection_running = false; std::mutex startup_mutex; }; diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 9f036a8a9b6..2c9834ae077 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -47,7 +47,16 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( for (size_t queue_id = 0; queue_id < num_queues; ++queue_id) bindQueue(queue_id); - consumer_channel->onReady([&]() { subscribe(); }); + consumer_channel->onReady([&]() + { + consumer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Consumer {} error: {}", consumer_tag, message); + channel_error.store(true); + }); + + subscribe(); + }); } @@ -62,16 +71,16 @@ void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) { std::atomic bindings_created = false, bindings_error = false; - auto success_callback = [&](const std::string & queue_name_, int msgcount, int /* consumercount */) + auto success_callback = [&](const std::string & queue_name, int msgcount, int /* consumercount */) { - queues.emplace_back(queue_name_); - LOG_DEBUG(log, "Queue " + queue_name_ + " is declared"); + queues.emplace_back(queue_name); + LOG_DEBUG(log, "Queue {} is declared", queue_name); if (msgcount) - LOG_TRACE(log, "Queue " + queue_name_ + " is non-empty. Non-consumed messaged will also be delivered."); + LOG_TRACE(log, "Queue {} is non-empty. Non-consumed messaged will also be delivered", queue_name); /// Binding key must be a string integer in case of hash exchange (here it is either hash or fanout). - setup_channel->bindQueue(exchange_name, queue_name_, std::to_string(channel_id)) + setup_channel->bindQueue(exchange_name, queue_name, std::to_string(channel_id)) .onSuccess([&] { bindings_created = true; @@ -114,22 +123,13 @@ void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) void ReadBufferFromRabbitMQConsumer::subscribe() { - count_subscribed = 0; for (const auto & queue_name : queues) { consumer_channel->consume(queue_name) .onSuccess([&](const std::string & consumer) { - ++count_subscribed; LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); - - consumer_error = false; consumer_tag = consumer; - - consumer_channel->onError([&](const char * message) - { - LOG_ERROR(log, "Consumer {} error: {}", consumer_tag, message); - }); }) .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) { @@ -144,36 +144,12 @@ void ReadBufferFromRabbitMQConsumer::subscribe() }) .onError([&](const char * message) { - consumer_error = true; LOG_ERROR(log, "Consumer {} failed. Reason: {}", channel_id, message); }); } } -void ReadBufferFromRabbitMQConsumer::checkSubscription() -{ - if (count_subscribed == num_queues || !consumer_channel->usable()) - return; - - wait_subscribed = num_queues; - - /// These variables are updated in a separate thread. - while (count_subscribed != wait_subscribed && !consumer_error) - { - iterateEventLoop(); - } - - LOG_TRACE(log, "Consumer {} is subscribed to {} queues", channel_id, count_subscribed); - - /// Updated in callbacks which are run by the loop. - if (count_subscribed == num_queues) - return; - - subscribe(); -} - - void ReadBufferFromRabbitMQConsumer::ackMessages() { UInt64 delivery_tag = last_inserted_delivery_tag; @@ -209,4 +185,26 @@ bool ReadBufferFromRabbitMQConsumer::nextImpl() return false; } + +void ReadBufferFromRabbitMQConsumer::restoreChannel(ChannelPtr new_channel) +{ + if (consumer_channel->usable()) + return; + + consumer_channel = std::move(new_channel); + consumer_channel->onReady([&]() + { + LOG_TRACE(log, "Channel {} is restored", channel_id); + channel_error.store(false); + consumer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Consumer {} error: {}", consumer_tag, message); + channel_error.store(true); + }); + + subscribe(); + }); +} + + } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index 6448389aea5..d3f560fad3b 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -46,7 +46,8 @@ public: }; void allowNext() { allowed = true; } // Allow to read next message. - void checkSubscription(); + bool channelUsable() { return !channel_error.load(); } + void restoreChannel(ChannelPtr new_channel); void updateNextDeliveryTag(UInt64 delivery_tag) { last_inserted_delivery_tag = delivery_tag; } void ackMessages(); @@ -71,15 +72,13 @@ private: const std::atomic & stopped; const String deadletter_exchange; - std::atomic consumer_error = false; - std::atomic count_subscribed = 0, wait_subscribed; + std::atomic channel_error = false; String consumer_tag; ConcurrentBoundedQueue received; UInt64 last_inserted_delivery_tag = 0, prev_tag = 0; MessageData current; std::vector queues; - std::unordered_map subscribed_queue; bool nextImpl() override; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index f31cf3f4f72..67f3daa81ec 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -39,7 +39,7 @@ namespace DB { static const auto CONNECT_SLEEP = 200; -static const auto RETRIES_MAX = 1000; +static const auto RETRIES_MAX = 20; static const auto HEARTBEAT_RESCHEDULE_MS = 3000; namespace ErrorCodes @@ -98,7 +98,6 @@ StorageRabbitMQ::StorageRabbitMQ( { loop = std::make_unique(); uv_loop_init(loop.get()); - event_handler = std::make_shared(loop.get(), log); connection = std::make_shared(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); @@ -138,16 +137,6 @@ StorageRabbitMQ::StorageRabbitMQ( exchange_type = AMQP::ExchangeType::fanout; } - if (exchange_type == AMQP::ExchangeType::headers) - { - for (const auto & header : routing_keys) - { - std::vector matching; - boost::split(matching, header, [](char c){ return c == '='; }); - bind_headers[matching[0]] = matching[1]; - } - } - auto table_id = getStorageID(); String table_name = table_id.table_name; @@ -163,7 +152,7 @@ StorageRabbitMQ::StorageRabbitMQ( void StorageRabbitMQ::heartbeatFunc() { - if (!stream_cancelled) + if (!stream_cancelled && event_handler->connectionRunning()) { LOG_TRACE(log, "Sending RabbitMQ heartbeat"); connection->heartbeat(); @@ -174,8 +163,11 @@ void StorageRabbitMQ::heartbeatFunc() void StorageRabbitMQ::loopingFunc() { - LOG_DEBUG(log, "Starting event looping iterations"); - event_handler->startLoop(); + if (event_handler->connectionRunning()) + { + LOG_DEBUG(log, "Starting event looping iterations"); + event_handler->startLoop(); + } } @@ -231,6 +223,14 @@ void StorageRabbitMQ::bindExchange() if (exchange_type == AMQP::ExchangeType::headers) { + AMQP::Table bind_headers; + for (const auto & header : routing_keys) + { + std::vector matching; + boost::split(matching, header, [](char c){ return c == '='; }); + bind_headers[matching[0]] = matching[1]; + } + setup_channel->bindExchange(exchange_name, bridge_exchange, routing_keys[0], bind_headers) .onSuccess([&]() { @@ -299,10 +299,66 @@ void StorageRabbitMQ::unbindExchange() event_handler->stop(); looping_task->deactivate(); + heartbeat_task->deactivate(); }); } +bool StorageRabbitMQ::restoreConnection() +{ + if (restore_connection.try_lock()) + { + /// This lock is to synchronize with getChannel(). + std::lock_guard lk(connection_mutex); + + if (!connection->usable() || !connection->ready()) + { + LOG_TRACE(log, "Trying to restore consumer connection"); + + if (!connection->closed()) + connection->close(); + + connection = std::make_shared(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); + + size_t cnt_retries = 0; + while (!connection->ready() && ++cnt_retries != RETRIES_MAX) + { + event_handler->iterateLoop(); + std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); + } + } + + if (event_handler->connectionRunning()) + { + LOG_TRACE(log, "Connection restored"); + + heartbeat_task->scheduleAfter(HEARTBEAT_RESCHEDULE_MS); + looping_task->activateAndSchedule(); + } + else + { + LOG_TRACE(log, "Connection refused"); + } + + restore_connection.unlock(); + } + else + { + std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); + } + + return event_handler->connectionRunning(); +} + + +ChannelPtr StorageRabbitMQ::getChannel() +{ + std::lock_guard lk(connection_mutex); + ChannelPtr new_channel = std::make_shared(connection.get()); + return new_channel; +} + + Pipes StorageRabbitMQ::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 9c7df1b1421..31e045ddb87 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -58,6 +58,10 @@ public: bool checkBridge() const { return !exchange_removed.load(); } void unbindExchange(); + bool connectionRunning() { return event_handler->connectionRunning(); } + bool restoreConnection(); + ChannelPtr getChannel(); + protected: StorageRabbitMQ( const StorageID & table_id_, @@ -109,11 +113,11 @@ private: String local_exchange, bridge_exchange, consumer_exchange; std::once_flag flag; - AMQP::Table bind_headers; size_t next_channel_id = 1; /// Must >= 1 because it is used as a binding key, which has to be > 0 bool update_channel_id = false; std::atomic loop_started = false, exchange_removed = false; ChannelPtr setup_channel; + std::mutex connection_mutex, restore_connection; BackgroundSchedulePool::TaskHolder streaming_task; BackgroundSchedulePool::TaskHolder heartbeat_task; From 62293f80c0a20144f260b5796ecf687fd8c3642d Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 3 Aug 2020 05:46:57 +0000 Subject: [PATCH 0029/1911] Small fixes --- .../RabbitMQ/RabbitMQBlockOutputStream.cpp | 2 +- src/Storages/RabbitMQ/RabbitMQHandler.cpp | 1 + .../ReadBufferFromRabbitMQConsumer.cpp | 2 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 5 +- .../WriteBufferToRabbitMQProducer.cpp | 253 +++++++++--------- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 10 +- 6 files changed, 139 insertions(+), 134 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index 37b39bbaeae..517b6bfaf68 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -63,7 +63,7 @@ void RabbitMQBlockOutputStream::writeSuffix() if (buffer) { buffer->updateMaxWait(); - buffer->finilizeProducer(); + buffer->commit(); } } diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.cpp b/src/Storages/RabbitMQ/RabbitMQHandler.cpp index ecaa109c184..c7186e3d3ff 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.cpp +++ b/src/Storages/RabbitMQ/RabbitMQHandler.cpp @@ -27,6 +27,7 @@ void RabbitMQHandler::onError(AMQP::TcpConnection * connection, const char * mes void RabbitMQHandler::onReady(AMQP::TcpConnection * /* connection */) { + LOG_TRACE(log, "Connection is ready"); connection_running.store(true); } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 2c9834ae077..cb3ef43d4d3 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -128,8 +128,8 @@ void ReadBufferFromRabbitMQConsumer::subscribe() consumer_channel->consume(queue_name) .onSuccess([&](const std::string & consumer) { - LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); consumer_tag = consumer; + LOG_TRACE(log, "Consumer {} (consumer tag: {}) is subscribed to queue {}", channel_id, consumer, queue_name); }) .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) { diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 67f3daa81ec..1e6e22c7c6e 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -109,7 +109,10 @@ StorageRabbitMQ::StorageRabbitMQ( } if (!connection->ready()) + { + uv_loop_close(loop.get()); throw Exception("Cannot set up connection for consumers", ErrorCodes::CANNOT_CONNECT_RABBITMQ); + } rabbitmq_context.makeQueryContext(); StorageInMemoryMetadata storage_metadata; @@ -498,7 +501,7 @@ ProducerBufferPtr StorageRabbitMQ::createWriteBuffer() { return std::make_shared( parsed_address, global_context, login_password, routing_keys, exchange_name, exchange_type, - log, num_consumers * num_queues, use_transactional_channel, persistent, + log, use_transactional_channel, persistent, row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); } diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index d74e94d74d2..ee8d8cf88da 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -20,8 +20,7 @@ namespace ErrorCodes static const auto QUEUE_SIZE = 50000; static const auto CONNECT_SLEEP = 200; -static const auto RETRIES_MAX = 1000; -static const auto LOOP_WAIT = 10; +static const auto RETRIES_MAX = 20; static const auto BATCH = 10000; WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( @@ -32,7 +31,6 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( const String & exchange_name_, const AMQP::ExchangeType exchange_type_, Poco::Logger * log_, - size_t num_queues_, const bool use_transactional_channel_, const bool persistent_, std::optional delimiter, @@ -44,10 +42,9 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( , routing_keys(routing_keys_) , exchange_name(exchange_name_) , exchange_type(exchange_type_) - , num_queues(num_queues_) , use_transactional_channel(use_transactional_channel_) , persistent(persistent_) - , payloads(QUEUE_SIZE * num_queues) + , payloads(QUEUE_SIZE) , log(log_) , delim(delimiter) , max_rows(rows_per_message) @@ -59,8 +56,8 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( event_handler = std::make_unique(loop.get(), log); /// New coonection for each publisher because cannot publish from different threads with the same connection.(https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) - setupConnection(0); - setupChannel(0); + setupConnection(); + setupChannel(); writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); writing_task->deactivate(); @@ -85,85 +82,6 @@ WriteBufferToRabbitMQProducer::~WriteBufferToRabbitMQProducer() } -void WriteBufferToRabbitMQProducer::setupConnection(bool remove_prev_connection) -{ - if (remove_prev_connection && connection) - { - connection->close(); - connection.release(); - } - - connection = std::make_unique(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); - - size_t cnt_retries = 0; - while (!connection->ready() && ++cnt_retries != RETRIES_MAX) - { - event_handler->iterateLoop(); - std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); - } - - if (!connection->ready()) - { - throw Exception("Cannot set up connection for producer", ErrorCodes::CANNOT_CONNECT_RABBITMQ); - } -} - - -void WriteBufferToRabbitMQProducer::setupChannel(bool remove_prev_channel) -{ - if (remove_prev_channel && producer_channel) - { - producer_channel->close(); - producer_channel.release(); - } - - producer_channel = std::make_unique(connection.get()); - producer_channel->onError([&](const char * message) - { - LOG_ERROR(log, "Prodcuer error: {}", message); - }); - - if (use_transactional_channel) - { - producer_channel->startTransaction(); - } - else - { - /// Same as here https://www.rabbitmq.com/blog/2011/02/10/introducing-publisher-confirms/ - remove_confirmed_tag = [&](uint64_t received_delivery_tag, bool multiple) - { - std::lock_guard lock(mutex); - auto found_tag_pos = delivery_tags_record.find(received_delivery_tag); - if (found_tag_pos != delivery_tags_record.end()) - { - if (multiple) - { - ++found_tag_pos; - delivery_tags_record.erase(delivery_tags_record.begin(), found_tag_pos); - } - else - delivery_tags_record.erase(found_tag_pos); - } - }; - - /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, it - * will be requed in returned_callback. If persistent == false, message is confirmed the moment it is enqueued. If fails, it is - * not requeued. First option is two times slower than the second, so default is second and the first is turned on in table setting. - */ - producer_channel->confirmSelect() - .onAck([&](uint64_t acked_delivery_tag, bool multiple) - { - remove_confirmed_tag(acked_delivery_tag, multiple); - }) - .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) - { - if (!persistent) - remove_confirmed_tag(nacked_delivery_tag, multiple); - }); - } -} - - void WriteBufferToRabbitMQProducer::countRow() { if (++rows % max_rows == 0) @@ -195,18 +113,100 @@ void WriteBufferToRabbitMQProducer::countRow() } +bool WriteBufferToRabbitMQProducer::setupConnection() +{ + connection = std::make_unique(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); + + size_t cnt_retries = 0; + while (!connection->ready() && ++cnt_retries != RETRIES_MAX) + { + event_handler->iterateLoop(); + std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); + } + + if (!connection->ready()) + return false; + + return true; +} + + +void WriteBufferToRabbitMQProducer::setupChannel() +{ + producer_channel = std::make_unique(connection.get()); + producer_channel->onError([&](const char * message) + { + /// Means channel ends up in an error state and is not usable anymore. + LOG_ERROR(log, "Producer error: {}", message); + producer_channel->close(); + }); + + producer_channel->onReady([&]() + { + LOG_TRACE(log, "Producer channel is ready"); + + if (use_transactional_channel) + { + producer_channel->startTransaction(); + } + else + { + /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, it + * will be requed in returned_callback. If persistent == false, message is confirmed the moment it is enqueued. If fails, it is + * not requeued. First option is two times slower than the second, so default is second and the first is turned on in table setting. + * Persistent message is not requeued if it is unroutable, i.e. no queues are bound to given exchange with the given routing key - + * this is a responsibility of a client. It can be requeued in this case if AMQP::mandatory is set, but it is pointless. Probably + */ + producer_channel->confirmSelect() + .onAck([&](uint64_t acked_delivery_tag, bool multiple) + { + removeConfirmed(acked_delivery_tag, multiple); + }) + .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) + { + if (!persistent) + removeConfirmed(nacked_delivery_tag, multiple); + }); + } + }); +} + + +void WriteBufferToRabbitMQProducer::removeConfirmed(UInt64 received_delivery_tag, bool multiple) +{ + /// Same as here https://www.rabbitmq.com/blog/2011/02/10/introducing-publisher-confirms/ + std::lock_guard lock(mutex); + auto found_tag_pos = delivery_tags_record.find(received_delivery_tag); + if (found_tag_pos != delivery_tags_record.end()) + { + /// If multiple is true, then all delivery tags up to and including current are confirmed. + if (multiple) + { + ++found_tag_pos; + delivery_tags_record.erase(delivery_tags_record.begin(), found_tag_pos); + LOG_TRACE(log, "Confirmed all delivery tags up to {}", received_delivery_tag); + } + else + { + delivery_tags_record.erase(found_tag_pos); + LOG_TRACE(log, "Confirmed delivery tag {}", received_delivery_tag); + } + } +} + + void WriteBufferToRabbitMQProducer::writingFunc() { String payload; UInt64 message_id = 0; - auto returned_callback = [&](const AMQP::Message & message, int16_t /* code */, const std::string & /* description */) + auto returned_callback = [&](const AMQP::Message & message, int16_t code, const std::string & description) { payloads.push(std::string(message.body(), message.size())); - //LOG_DEBUG(log, "Message returned with code: {}, description: {}. Republishing", code, description); + LOG_DEBUG(log, "Message returned with code: {}, description: {}. Republishing", code, description); }; - while ((!payloads.empty() || wait_all) && connection->usable()) + while (!payloads.empty() || wait_all) { while (!payloads.empty() && producer_channel->usable()) { @@ -242,7 +242,7 @@ void WriteBufferToRabbitMQProducer::writingFunc() if (wait_num.load() && delivery_tags_record.empty()) { wait_all.store(false); - LOG_DEBUG(log, "All messages are successfully published"); + LOG_TRACE(log, "All messages are successfully published"); } else { @@ -252,57 +252,60 @@ void WriteBufferToRabbitMQProducer::writingFunc() /// Most channel based errors result in channel closure, which is very likely to trigger connection closure. if (connection->usable() && connection->ready() && !producer_channel->usable()) { - LOG_DEBUG(log, "Channel is not usable. Creating a new one"); - setupChannel(1); + LOG_TRACE(log, "Channel is not usable. Creating a new one"); + setupChannel(); } else if (!connection->usable() || !connection->ready()) { - LOG_DEBUG(log, "Connection is not usable. Creating a new one"); - setupConnection(1); - setupChannel(1); + LOG_TRACE(log, "Trying to restore connection"); + + if (setupConnection()) + { + LOG_TRACE(log, "Connection restored. Creating a channel"); + setupChannel(); + } + + LOG_DEBUG(log, "Currently {} messages have not been confirmed yet, {} messages are waiting to be published", delivery_tags_record.size(), payloads.size()); } } - - LOG_DEBUG(log, "Delivered messages"); } -void WriteBufferToRabbitMQProducer::finilizeProducer() +void WriteBufferToRabbitMQProducer::commit() { - if (use_transactional_channel) + if (!use_transactional_channel) + return; + + std::atomic answer_received = false, wait_rollback = false; + producer_channel->commitTransaction() + .onSuccess([&]() { - std::atomic answer_received = false, wait_rollback = false; - producer_channel->commitTransaction() + answer_received = true; + wait_all.store(false); + LOG_TRACE(log, "All messages were successfully published"); + }) + .onError([&](const char * message1) + { + answer_received = true; + wait_all.store(false); + LOG_TRACE(log, "Publishing not successful: {}", message1); + + wait_rollback = true; + producer_channel->rollbackTransaction() .onSuccess([&]() { - answer_received = true; - wait_all.store(false); - LOG_TRACE(log, "All messages were successfully published"); + wait_rollback = false; }) - .onError([&](const char * message1) + .onError([&](const char * message2) { - answer_received = true; - wait_all.store(false); - wait_rollback = true; - LOG_TRACE(log, "Publishing not successful: {}", message1); - producer_channel->rollbackTransaction() - .onSuccess([&]() - { - wait_rollback = false; - }) - .onError([&](const char * message2) - { - LOG_ERROR(log, "Failed to rollback transaction: {}", message2); - wait_rollback = false; - }); + LOG_ERROR(log, "Failed to rollback transaction: {}", message2); + wait_rollback = false; }); + }); - size_t count_retries = 0; - while ((!answer_received || wait_rollback) && ++count_retries != RETRIES_MAX) - { - iterateEventLoop(); - std::this_thread::sleep_for(std::chrono::milliseconds(LOOP_WAIT)); - } + while (!answer_received || wait_rollback) + { + iterateEventLoop(); } } diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 188bd5676f4..9b809c1af81 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -25,7 +25,6 @@ public: const String & exchange_name_, const AMQP::ExchangeType exchange_type_, Poco::Logger * log_, - size_t num_queues_, const bool use_transactional_channel_, const bool persistent_, std::optional delimiter, @@ -37,22 +36,22 @@ public: void countRow(); void activateWriting() { writing_task->activateAndSchedule(); } - void finilizeProducer(); + void commit(); void updateMaxWait() { wait_num.store(delivery_tag); } private: void nextImpl() override; void iterateEventLoop(); void writingFunc(); - void setupConnection(bool remove_prev_connection); - void setupChannel(bool remove_prev_channel); + bool setupConnection(); + void setupChannel(); + void removeConfirmed(UInt64 received_delivery_tag, bool multiple); std::pair parsed_address; const std::pair login_password; const Names routing_keys; const String exchange_name; AMQP::ExchangeType exchange_type; - const size_t num_queues; const bool use_transactional_channel; const bool persistent; @@ -70,7 +69,6 @@ private: std::atomic wait_num = 0; std::set delivery_tags_record; std::mutex mutex; - std::function remove_confirmed_tag; Poco::Logger * log; const std::optional delim; From d5b1332b6717485f7b5c6ca08f454aa8dc775d86 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 2 Aug 2020 19:30:55 +0000 Subject: [PATCH 0030/1911] Stop publish untill batch is confirmed --- .../WriteBufferToRabbitMQProducer.cpp | 181 ++++++++++-------- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 7 +- .../integration/test_storage_rabbitmq/test.py | 4 +- 3 files changed, 110 insertions(+), 82 deletions(-) diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index ee8d8cf88da..883ee70f5d5 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -18,10 +18,9 @@ namespace ErrorCodes extern const int CANNOT_CONNECT_RABBITMQ; } -static const auto QUEUE_SIZE = 50000; static const auto CONNECT_SLEEP = 200; static const auto RETRIES_MAX = 20; -static const auto BATCH = 10000; +static const auto BATCH = 512; WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( std::pair & parsed_address_, @@ -44,7 +43,8 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( , exchange_type(exchange_type_) , use_transactional_channel(use_transactional_channel_) , persistent(persistent_) - , payloads(QUEUE_SIZE) + , payloads(BATCH) + , returned(BATCH << 6) , log(log_) , delim(delimiter) , max_rows(rows_per_message) @@ -56,8 +56,8 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( event_handler = std::make_unique(loop.get(), log); /// New coonection for each publisher because cannot publish from different threads with the same connection.(https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) - setupConnection(); - setupChannel(); + if (setupConnection()) + setupChannel(); writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); writing_task->deactivate(); @@ -104,11 +104,8 @@ void WriteBufferToRabbitMQProducer::countRow() chunks.clear(); set(nullptr, 0); - ++delivery_tag; payloads.push(payload); - - std::lock_guard lock(mutex); - delivery_tags_record.insert(delivery_tags_record.end(), delivery_tag); + ++payload_counter; } } @@ -117,7 +114,9 @@ bool WriteBufferToRabbitMQProducer::setupConnection() { connection = std::make_unique(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); + LOG_TRACE(log, "Trying to set up connection"); size_t cnt_retries = 0; + while (!connection->ready() && ++cnt_retries != RETRIES_MAX) { event_handler->iterateLoop(); @@ -136,14 +135,20 @@ void WriteBufferToRabbitMQProducer::setupChannel() producer_channel = std::make_unique(connection.get()); producer_channel->onError([&](const char * message) { + LOG_DEBUG(log, "Producer error: {}. Currently {} messages have not been confirmed yet, {} messages are waiting to be published", + message, delivery_tags_record.size(), payloads.size()); + /// Means channel ends up in an error state and is not usable anymore. - LOG_ERROR(log, "Producer error: {}", message); producer_channel->close(); }); producer_channel->onReady([&]() { - LOG_TRACE(log, "Producer channel is ready"); + LOG_DEBUG(log, "Producer channel is ready"); + + /// Delivery tags are scoped per channel. + delivery_tags_record.clear(); + delivery_tag = 0; if (use_transactional_channel) { @@ -151,11 +156,11 @@ void WriteBufferToRabbitMQProducer::setupChannel() } else { - /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, it - * will be requed in returned_callback. If persistent == false, message is confirmed the moment it is enqueued. If fails, it is - * not requeued. First option is two times slower than the second, so default is second and the first is turned on in table setting. - * Persistent message is not requeued if it is unroutable, i.e. no queues are bound to given exchange with the given routing key - - * this is a responsibility of a client. It can be requeued in this case if AMQP::mandatory is set, but it is pointless. Probably + /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, + * it will be requed in returned_callback. If persistent == false, message is confirmed the moment it is enqueued. If fails, it + * is not requeued. First option is two times slower than the second, so default is second and the first is turned on in table + * setting. Persistent message is not requeued if it is unroutable, i.e. no queues are bound to given exchange with the given + * routing key - this is a responsibility of a client. It can be requeued in this case if AMQP::mandatory is set, but pointless. */ producer_channel->confirmSelect() .onAck([&](uint64_t acked_delivery_tag, bool multiple) @@ -184,90 +189,110 @@ void WriteBufferToRabbitMQProducer::removeConfirmed(UInt64 received_delivery_tag { ++found_tag_pos; delivery_tags_record.erase(delivery_tags_record.begin(), found_tag_pos); - LOG_TRACE(log, "Confirmed all delivery tags up to {}", received_delivery_tag); + //LOG_DEBUG(log, "Confirmed all delivery tags up to {}", received_delivery_tag); } else { delivery_tags_record.erase(found_tag_pos); - LOG_TRACE(log, "Confirmed delivery tag {}", received_delivery_tag); + //LOG_DEBUG(log, "Confirmed delivery tag {}", received_delivery_tag); } } } -void WriteBufferToRabbitMQProducer::writingFunc() +void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue & messages) { String payload; - UInt64 message_id = 0; - - auto returned_callback = [&](const AMQP::Message & message, int16_t code, const std::string & description) + while (!messages.empty()) { - payloads.push(std::string(message.body(), message.size())); + messages.pop(payload); + AMQP::Envelope envelope(payload.data(), payload.size()); + + /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse. + if (persistent) + envelope.setDeliveryMode(2); + + if (exchange_type == AMQP::ExchangeType::consistent_hash) + { + producer_channel->publish(exchange_name, std::to_string(delivery_tag), envelope).onReturned(returned_callback); + } + else if (exchange_type == AMQP::ExchangeType::headers) + { + envelope.setHeaders(key_arguments); + producer_channel->publish(exchange_name, "", envelope).onReturned(returned_callback); + } + else + { + producer_channel->publish(exchange_name, routing_keys[0], envelope).onReturned(returned_callback); + } + + if (producer_channel->usable()) + { + ++delivery_tag; + delivery_tags_record.insert(delivery_tags_record.end(), delivery_tag); + + if (delivery_tag % BATCH == 0) + break; + } + else + { + break; + } + } + + iterateEventLoop(); +} + +/* Currently implemented “asynchronous publisher confirms” - does not stop after each publish to wait for each individual confirm. An + * asynchronous publisher may have any number of messages in-flight (unconfirmed) at a time. + * Synchronous publishing is where after each publish need to wait for the acknowledgement (ack/nack - see confirmSelect() in channel + * declaration), which is very slow because takes starting event loop and waiting for corresponding callback - can really take a while. + * + * Async publishing works well in all failure cases except for connection failure, because if connection fails - not all Ack/Nack might be + * receieved from the server (and even if all messages were successfully delivered, publisher will not be able to know it). Also in this + * case onReturned callback will not be received, so loss is possible for messages that were published but have not received confirm from + * server before connection loss, because then publisher won't know if message was delivered or not. + * + * To make it a delivery with no loss and minimal possible amount of duplicates - need to use synchronous publishing (which is too slow). + * With async publishing at-least-once delivery is achieved with (batch) publishing and manual republishing in case when not all delivery + * tags were confirmed (ack/nack) before connection loss. Here the maximum number of possible duplicates is no more than batch size. + * (Manual last batch republishing is only for case of connection loss, in all other failure cases - onReturned callback will be received.) + * + * So currently implemented async batch publishing, but for now without manual republishing (because still in doubt how to do it nicely, + * but current idea is to store in delivery_tags_record not just delivery tags, but pair: (delivery_tag, message). As currently once the + * publisher receives acknowledgement from the server that the message was sucessfully delivered - a "confirmListener" will delete its + * delivery tag from the set of pending acknowledgemens, then we can as well delete the payload. If connection fails, undeleted delivery + * tags indicate messages, whose fate is unknown, so corresponding payloads should be republished.) +*/ +void WriteBufferToRabbitMQProducer::writingFunc() +{ + returned_callback = [&](const AMQP::Message & message, int16_t code, const std::string & description) + { + returned.tryPush(std::string(message.body(), message.size())); LOG_DEBUG(log, "Message returned with code: {}, description: {}. Republishing", code, description); + + /* Here can be added a value to AMQP::Table field of AMQP::Envelope (and then it should be queue instead of + * queue) - to indicate that message was republished. Later a consumer will be able to extract this field and understand + * that this message was republished and can probably be a duplicate (as RabbitMQ does not guarantee exactly-once delivery). + */ }; while (!payloads.empty() || wait_all) { - while (!payloads.empty() && producer_channel->usable()) - { - payloads.pop(payload); - AMQP::Envelope envelope(payload.data(), payload.size()); + if (!returned.empty() && producer_channel->usable()) + publish(returned); + else if (!payloads.empty() && delivery_tags_record.empty() && producer_channel->usable()) + publish(payloads); - ++message_id; - if (wait_num) - message_id %= wait_num; + iterateEventLoop(); - /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse. - if (persistent) - envelope.setDeliveryMode(2); - - if (exchange_type == AMQP::ExchangeType::consistent_hash) - { - producer_channel->publish(exchange_name, std::to_string(message_id), envelope).onReturned(returned_callback); - } - else if (exchange_type == AMQP::ExchangeType::headers) - { - envelope.setHeaders(key_arguments); - producer_channel->publish(exchange_name, "", envelope).onReturned(returned_callback); - } - else - { - producer_channel->publish(exchange_name, routing_keys[0], envelope).onReturned(returned_callback); - } - - if (message_id % BATCH == 0) - iterateEventLoop(); - } - - if (wait_num.load() && delivery_tags_record.empty()) - { + if (wait_num.load() && delivery_tags_record.empty() && payloads.empty()) wait_all.store(false); - LOG_TRACE(log, "All messages are successfully published"); - } - else - { - iterateEventLoop(); - } - - /// Most channel based errors result in channel closure, which is very likely to trigger connection closure. - if (connection->usable() && connection->ready() && !producer_channel->usable()) - { - LOG_TRACE(log, "Channel is not usable. Creating a new one"); + else if ((!producer_channel->usable() && connection->usable()) || (!connection->usable() && setupConnection())) setupChannel(); - } - else if (!connection->usable() || !connection->ready()) - { - LOG_TRACE(log, "Trying to restore connection"); - - if (setupConnection()) - { - LOG_TRACE(log, "Connection restored. Creating a channel"); - setupChannel(); - } - - LOG_DEBUG(log, "Currently {} messages have not been confirmed yet, {} messages are waiting to be published", delivery_tags_record.size(), payloads.size()); - } } + + LOG_DEBUG(log, "Processing ended"); } diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 9b809c1af81..d8e3db37043 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -37,7 +37,7 @@ public: void countRow(); void activateWriting() { writing_task->activateAndSchedule(); } void commit(); - void updateMaxWait() { wait_num.store(delivery_tag); } + void updateMaxWait() { wait_num.store(payload_counter); } private: void nextImpl() override; @@ -46,6 +46,7 @@ private: bool setupConnection(); void setupChannel(); void removeConfirmed(UInt64 received_delivery_tag, bool multiple); + void publish(ConcurrentBoundedQueue & message); std::pair parsed_address; const std::pair login_password; @@ -63,12 +64,14 @@ private: std::unique_ptr connection; std::unique_ptr producer_channel; - ConcurrentBoundedQueue payloads; + ConcurrentBoundedQueue payloads, returned; UInt64 delivery_tag = 0; std::atomic wait_all = true; std::atomic wait_num = 0; std::set delivery_tags_record; std::mutex mutex; + UInt64 payload_counter = 0; + std::function returned_callback; Poco::Logger * log; const std::optional delim; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index bc4585fb6f2..e45afa47425 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -1382,8 +1382,8 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): for consumer_id in range(num_tables_to_receive + num_tables_to_ignore): instance.query(''' - DROP TABLE IF EXISTS test.direct_exchange_{0}; - DROP TABLE IF EXISTS test.direct_exchange_{0}_mv; + DROP TABLE IF EXISTS test.headers_exchange_{0}_mv; + DROP TABLE IF EXISTS test.headers_exchange_{0}; '''.format(consumer_id)) instance.query(''' From 053f31cb77235e4da3d3401f64b24cb3b4cfc413 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 4 Aug 2020 15:13:09 +0000 Subject: [PATCH 0031/1911] Better confirmListener --- .../WriteBufferToRabbitMQProducer.cpp | 145 ++++++++---------- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 8 +- 2 files changed, 69 insertions(+), 84 deletions(-) diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 883ee70f5d5..c2ab8e3e843 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -13,11 +13,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_CONNECT_RABBITMQ; -} - static const auto CONNECT_SLEEP = 200; static const auto RETRIES_MAX = 20; static const auto BATCH = 512; @@ -133,23 +128,29 @@ bool WriteBufferToRabbitMQProducer::setupConnection() void WriteBufferToRabbitMQProducer::setupChannel() { producer_channel = std::make_unique(connection.get()); + producer_channel->onError([&](const char * message) { - LOG_DEBUG(log, "Producer error: {}. Currently {} messages have not been confirmed yet, {} messages are waiting to be published", - message, delivery_tags_record.size(), payloads.size()); + LOG_ERROR(log, "Producer error: {}", message); /// Means channel ends up in an error state and is not usable anymore. producer_channel->close(); + + for (auto record = delivery_record.begin(); record != delivery_record.end(); record++) + returned.tryPush(record->second); + + LOG_DEBUG(log, "Currently {} messages have not been confirmed yet, {} waiting to be published, {} will be republished", + delivery_record.size(), payloads.size(), returned.size()); + + /// Delivery tags are scoped per channel. + delivery_record.clear(); + delivery_tag = 0; }); producer_channel->onReady([&]() { LOG_DEBUG(log, "Producer channel is ready"); - /// Delivery tags are scoped per channel. - delivery_tags_record.clear(); - delivery_tag = 0; - if (use_transactional_channel) { producer_channel->startTransaction(); @@ -157,56 +158,76 @@ void WriteBufferToRabbitMQProducer::setupChannel() else { /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, - * it will be requed in returned_callback. If persistent == false, message is confirmed the moment it is enqueued. If fails, it - * is not requeued. First option is two times slower than the second, so default is second and the first is turned on in table - * setting. Persistent message is not requeued if it is unroutable, i.e. no queues are bound to given exchange with the given - * routing key - this is a responsibility of a client. It can be requeued in this case if AMQP::mandatory is set, but pointless. + * onNack() is received. If persistent == false, message is confirmed the moment it is enqueued. First option is two times + * slower than the second, so default is second and the first is turned on in table setting. + * + * "Publisher confirms" are implemented similar to strategy#3 here https://www.rabbitmq.com/tutorials/tutorial-seven-java.html */ producer_channel->confirmSelect() .onAck([&](uint64_t acked_delivery_tag, bool multiple) { - removeConfirmed(acked_delivery_tag, multiple); + removeConfirmed(acked_delivery_tag, multiple, false); }) .onNack([&](uint64_t nacked_delivery_tag, bool multiple, bool /* requeue */) { - if (!persistent) - removeConfirmed(nacked_delivery_tag, multiple); + removeConfirmed(nacked_delivery_tag, multiple, true); }); } }); } -void WriteBufferToRabbitMQProducer::removeConfirmed(UInt64 received_delivery_tag, bool multiple) +void WriteBufferToRabbitMQProducer::removeConfirmed(UInt64 received_delivery_tag, bool multiple, bool republish) { - /// Same as here https://www.rabbitmq.com/blog/2011/02/10/introducing-publisher-confirms/ - std::lock_guard lock(mutex); - auto found_tag_pos = delivery_tags_record.find(received_delivery_tag); - if (found_tag_pos != delivery_tags_record.end()) + auto record_iter = delivery_record.find(received_delivery_tag); + + if (record_iter != delivery_record.end()) { - /// If multiple is true, then all delivery tags up to and including current are confirmed. if (multiple) { - ++found_tag_pos; - delivery_tags_record.erase(delivery_tags_record.begin(), found_tag_pos); + /// If multiple is true, then all delivery tags up to and including current are confirmed (with ack or nack). + ++record_iter; + + if (republish) + for (auto record = delivery_record.begin(); record != record_iter; ++record) + returned.tryPush(record->second); + + /// Delete the records even in case when republished because new delivery tags will be assigned by the server. + delivery_record.erase(delivery_record.begin(), record_iter); + //LOG_DEBUG(log, "Confirmed all delivery tags up to {}", received_delivery_tag); } else { - delivery_tags_record.erase(found_tag_pos); + if (republish) + returned.tryPush(record_iter->second); + + delivery_record.erase(record_iter); + //LOG_DEBUG(log, "Confirmed delivery tag {}", received_delivery_tag); } } + /// else is theoretically not possible } -void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue & messages) +void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue & messages, bool republishing) { String payload; - while (!messages.empty()) + while (!messages.empty() && producer_channel->usable()) { messages.pop(payload); AMQP::Envelope envelope(payload.data(), payload.size()); + AMQP::Table message_settings = key_arguments; + + /* There is the case when connection is lost in the period after some messages were published and before ack/nack was sent by the + * server, then it means that publisher will never now whether those messages were delivered or not, and therefore those records + * that received no ack/nack before connection loss will be republished, so there might be duplicates. To let consumer know that + * received message might be a possible duplicate - a "republished" field is added to message metadata. + */ + message_settings["republished"] = std::to_string(republishing); + + envelope.setHeaders(message_settings); /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse. if (persistent) @@ -214,79 +235,45 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue & mes if (exchange_type == AMQP::ExchangeType::consistent_hash) { - producer_channel->publish(exchange_name, std::to_string(delivery_tag), envelope).onReturned(returned_callback); + producer_channel->publish(exchange_name, std::to_string(delivery_tag), envelope); } else if (exchange_type == AMQP::ExchangeType::headers) { - envelope.setHeaders(key_arguments); - producer_channel->publish(exchange_name, "", envelope).onReturned(returned_callback); + producer_channel->publish(exchange_name, "", envelope); } else { - producer_channel->publish(exchange_name, routing_keys[0], envelope).onReturned(returned_callback); + producer_channel->publish(exchange_name, routing_keys[0], envelope); } - if (producer_channel->usable()) - { - ++delivery_tag; - delivery_tags_record.insert(delivery_tags_record.end(), delivery_tag); + ++delivery_tag; + delivery_record.insert(delivery_record.end(), {delivery_tag, payload}); - if (delivery_tag % BATCH == 0) - break; - } - else - { + /// Need to break to let event loop run, because no publishing actually happend before looping. + if (delivery_tag % BATCH == 0) break; - } } iterateEventLoop(); } -/* Currently implemented “asynchronous publisher confirms” - does not stop after each publish to wait for each individual confirm. An - * asynchronous publisher may have any number of messages in-flight (unconfirmed) at a time. - * Synchronous publishing is where after each publish need to wait for the acknowledgement (ack/nack - see confirmSelect() in channel - * declaration), which is very slow because takes starting event loop and waiting for corresponding callback - can really take a while. - * - * Async publishing works well in all failure cases except for connection failure, because if connection fails - not all Ack/Nack might be - * receieved from the server (and even if all messages were successfully delivered, publisher will not be able to know it). Also in this - * case onReturned callback will not be received, so loss is possible for messages that were published but have not received confirm from - * server before connection loss, because then publisher won't know if message was delivered or not. - * - * To make it a delivery with no loss and minimal possible amount of duplicates - need to use synchronous publishing (which is too slow). - * With async publishing at-least-once delivery is achieved with (batch) publishing and manual republishing in case when not all delivery - * tags were confirmed (ack/nack) before connection loss. Here the maximum number of possible duplicates is no more than batch size. - * (Manual last batch republishing is only for case of connection loss, in all other failure cases - onReturned callback will be received.) - * - * So currently implemented async batch publishing, but for now without manual republishing (because still in doubt how to do it nicely, - * but current idea is to store in delivery_tags_record not just delivery tags, but pair: (delivery_tag, message). As currently once the - * publisher receives acknowledgement from the server that the message was sucessfully delivered - a "confirmListener" will delete its - * delivery tag from the set of pending acknowledgemens, then we can as well delete the payload. If connection fails, undeleted delivery - * tags indicate messages, whose fate is unknown, so corresponding payloads should be republished.) -*/ + void WriteBufferToRabbitMQProducer::writingFunc() { - returned_callback = [&](const AMQP::Message & message, int16_t code, const std::string & description) - { - returned.tryPush(std::string(message.body(), message.size())); - LOG_DEBUG(log, "Message returned with code: {}, description: {}. Republishing", code, description); - - /* Here can be added a value to AMQP::Table field of AMQP::Envelope (and then it should be queue instead of - * queue) - to indicate that message was republished. Later a consumer will be able to extract this field and understand - * that this message was republished and can probably be a duplicate (as RabbitMQ does not guarantee exactly-once delivery). - */ - }; - while (!payloads.empty() || wait_all) { + /* Publish main paylods only when there are no returned messages. This way it is ensured that returned.queue never grows too big + * and returned messages are republished as fast as possible. Also payloads.queue is fixed size and push attemt would block thread + * in countRow() once there is no space - that is intended. + */ if (!returned.empty() && producer_channel->usable()) - publish(returned); - else if (!payloads.empty() && delivery_tags_record.empty() && producer_channel->usable()) - publish(payloads); + publish(returned, true); + else if (!payloads.empty() && producer_channel->usable()) + publish(payloads, false); iterateEventLoop(); - if (wait_num.load() && delivery_tags_record.empty() && payloads.empty()) + if (wait_num.load() && delivery_record.empty() && payloads.empty() && returned.empty()) wait_all.store(false); else if ((!producer_channel->usable() && connection->usable()) || (!connection->usable() && setupConnection())) setupChannel(); diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index d8e3db37043..b9378695d8d 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -45,8 +45,8 @@ private: void writingFunc(); bool setupConnection(); void setupChannel(); - void removeConfirmed(UInt64 received_delivery_tag, bool multiple); - void publish(ConcurrentBoundedQueue & message); + void removeConfirmed(UInt64 received_delivery_tag, bool multiple, bool republish); + void publish(ConcurrentBoundedQueue & message, bool republishing); std::pair parsed_address; const std::pair login_password; @@ -68,10 +68,8 @@ private: UInt64 delivery_tag = 0; std::atomic wait_all = true; std::atomic wait_num = 0; - std::set delivery_tags_record; - std::mutex mutex; UInt64 payload_counter = 0; - std::function returned_callback; + std::map delivery_record; Poco::Logger * log; const std::optional delim; From 24b032b3786f350a77f32871e6f36c6a81ca13ce Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 6 Aug 2020 13:33:46 +0000 Subject: [PATCH 0032/1911] Allow multiple consumers for same queues --- .../ReadBufferFromRabbitMQConsumer.cpp | 24 ++--- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 31 ++++-- src/Storages/RabbitMQ/StorageRabbitMQ.h | 4 +- .../integration/test_storage_rabbitmq/test.py | 96 +++++++++++++++++-- 4 files changed, 126 insertions(+), 29 deletions(-) diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index cb3ef43d4d3..47c15df3bd3 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -79,7 +79,10 @@ void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) if (msgcount) LOG_TRACE(log, "Queue {} is non-empty. Non-consumed messaged will also be delivered", queue_name); - /// Binding key must be a string integer in case of hash exchange (here it is either hash or fanout). + /* Here we bind either to sharding exchange (consistent-hash) or to bridge exchange (fanout). All bindings to routing keys are + * done between client's exchange and local bridge exchange. Binding key must be a string integer in case of hash exchange, for + * fanout exchange it can be arbitrary. + */ setup_channel->bindQueue(exchange_name, queue_name, std::to_string(channel_id)) .onSuccess([&] { @@ -104,15 +107,11 @@ void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) queue_settings["x-dead-letter-exchange"] = deadletter_exchange; } - if (!queue_base.empty()) - { - const String queue_name = !hash_exchange ? queue_base : queue_base + "_" + std::to_string(channel_id) + "_" + std::to_string(queue_id); - setup_channel->declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); - } - else - { - setup_channel->declareQueue(AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); - } + /* The first option not just simplifies queue_name, but also implements the possibility to be able to resume reading from one + * specific queue when its name is specified in queue_base setting. + */ + const String queue_name = !hash_exchange ? queue_base : queue_base + "_" + std::to_string(channel_id) + "_" + std::to_string(queue_id); + setup_channel->declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); while (!bindings_created && !bindings_error) { @@ -128,8 +127,9 @@ void ReadBufferFromRabbitMQConsumer::subscribe() consumer_channel->consume(queue_name) .onSuccess([&](const std::string & consumer) { - consumer_tag = consumer; - LOG_TRACE(log, "Consumer {} (consumer tag: {}) is subscribed to queue {}", channel_id, consumer, queue_name); + if (consumer_tag.empty()) + consumer_tag = consumer; + LOG_TRACE(log, "Consumer {} is subscribed to queue {}, consumer tag {}", channel_id, queue_name, consumer); }) .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) { diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 1e6e22c7c6e..80f66c6be0d 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -143,9 +143,28 @@ StorageRabbitMQ::StorageRabbitMQ( auto table_id = getStorageID(); String table_name = table_id.table_name; - /// Make sure that local exchange name is unique for each table and is not the same as client's exchange name - local_exchange = exchange_name + "_" + table_name; - bridge_exchange = local_exchange + "_bridge"; + if (queue_base.empty()) + { + /// Make sure that local exchange name is unique for each table and is not the same as client's exchange name + sharding_exchange = exchange_name + "_" + table_name; + + /* By default without a specified queue name in queue's declaration - its name will be generated by the library, but its better + * to specify it unique for each table to reuse them once the table is recreated. So it means that queues remain the same for every + * table unless queue_base table setting is specified (which allows to register consumers to specific queues). Now this is a base + * for the names of later declared queue (as everything is based on names). + */ + queue_base = "queue_" + table_name; + } + else + { + /* In case different tables are used to register multiple consumers to the same queues (so queues are shared between tables) and + * at the same time sharding exchange is needed (if there are multiple shared queues), then those tables also need + * to share sharding exchange. + */ + sharding_exchange = exchange_name + queue_base; + } + + bridge_exchange = sharding_exchange + "_bridge"; /// One looping task for all consumers as they share the same connection == the same handler == the same event loop looping_task = global_context.getSchedulePool().createTask("RabbitMQLoopingTask", [this]{ loopingFunc(); }); @@ -203,19 +222,19 @@ void StorageRabbitMQ::initExchange() AMQP::Table binding_arguments; binding_arguments["hash-property"] = "message_id"; - setup_channel->declareExchange(local_exchange, AMQP::consistent_hash, AMQP::durable + AMQP::autodelete, binding_arguments) + setup_channel->declareExchange(sharding_exchange, AMQP::consistent_hash, AMQP::durable + AMQP::autodelete, binding_arguments) .onError([&](const char * message) { throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); }); - setup_channel->bindExchange(bridge_exchange, local_exchange, routing_keys[0]) + setup_channel->bindExchange(bridge_exchange, sharding_exchange, routing_keys[0]) .onError([&](const char * message) { throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); }); - consumer_exchange = local_exchange; + consumer_exchange = sharding_exchange; } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 31e045ddb87..5aa030d821c 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -95,7 +95,7 @@ private: bool hash_exchange; size_t num_queues; const bool use_transactional_channel; - const String queue_base; + String queue_base; const String deadletter_exchange; const bool persistent; @@ -111,7 +111,7 @@ private: std::mutex mutex; std::vector buffers; /// available buffers for RabbitMQ consumers - String local_exchange, bridge_exchange, consumer_exchange; + String sharding_exchange, bridge_exchange, consumer_exchange; std::once_flag flag; size_t next_channel_id = 1; /// Must >= 1 because it is used as a binding key, which has to be > 0 bool update_channel_id = false; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index e45afa47425..be45298b52f 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -512,8 +512,6 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): SELECT *, _consumer_tag AS consumer_tag FROM test.rabbitmq; ''') - time.sleep(1) - i = [0] messages_num = 10000 @@ -1546,7 +1544,7 @@ def test_rabbitmq_queue_resume_1(rabbitmq_cluster): ''') i = [0] - messages_num = 5000 + messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) @@ -1635,7 +1633,7 @@ def test_rabbitmq_queue_resume_2(rabbitmq_cluster): ''') i = [0] - messages_num = 5000 + messages_num = 10000 credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) @@ -1689,8 +1687,6 @@ def test_rabbitmq_queue_resume_2(rabbitmq_cluster): if int(result1) > collected: break - result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") - instance.query(''' DROP TABLE IF EXISTS test.rabbitmq_queue_resume; DROP TABLE IF EXISTS test.consumer; @@ -1698,7 +1694,6 @@ def test_rabbitmq_queue_resume_2(rabbitmq_cluster): ''') assert int(result1) > collected, 'ClickHouse lost some messages: {}'.format(result) - assert int(result2) == 2 @pytest.mark.timeout(420) @@ -1778,8 +1773,6 @@ def test_rabbitmq_consumer_acknowledgements(rabbitmq_cluster): if int(result1) >= messages_num * threads_num: break - #result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") - instance.query(''' DROP TABLE IF EXISTS test.rabbitmq_consumer_acks; DROP TABLE IF EXISTS test.consumer; @@ -1790,6 +1783,91 @@ def test_rabbitmq_consumer_acknowledgements(rabbitmq_cluster): assert int(result1) >= messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) +@pytest.mark.timeout(420) +def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): + instance.query(''' + DROP TABLE IF EXISTS test.destination; + CREATE TABLE test.destination(key UInt64, value UInt64, consumer_tag String) + ENGINE = MergeTree() + ORDER BY key; + ''') + + num_tables = 4 + for table_id in range(num_tables): + print("Setting up table {}".format(table_id)) + instance.query(''' + DROP TABLE IF EXISTS test.many_consumers_{0}; + DROP TABLE IF EXISTS test.many_consumers_{0}_mv; + CREATE TABLE test.many_consumers_{0} (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'many_consumers', + rabbitmq_num_queues = 2, + rabbitmq_num_consumers = 2, + rabbitmq_queue_base = 'many_consumers', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + CREATE MATERIALIZED VIEW test.many_consumers_{0}_mv TO test.destination AS + SELECT key, value, _consumer_tag as consumer_tag FROM test.many_consumers_{0}; + '''.format(table_id)) + + i = [0] + messages_num = 1000 + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + def produce(): + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + + messages = [] + for _ in range(messages_num): + messages.append(json.dumps({'key': i[0], 'value': i[0]})) + i[0] += 1 + current = 0 + for message in messages: + current += 1 + mes_id = str(current) + channel.basic_publish(exchange='many_consumers', routing_key='', + properties=pika.BasicProperties(message_id=mes_id), body=message) + connection.close() + + threads = [] + threads_num = 20 + + for _ in range(threads_num): + threads.append(threading.Thread(target=produce)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + result1 = '' + while True: + result1 = instance.query('SELECT count() FROM test.destination') + time.sleep(1) + if int(result1) == messages_num * threads_num: + break + + result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.destination") + + for thread in threads: + thread.join() + + for consumer_id in range(num_tables): + instance.query(''' + DROP TABLE IF EXISTS test.many_consumers_{0}; + DROP TABLE IF EXISTS test.many_consumers_{0}_mv; + '''.format(consumer_id)) + + instance.query(''' + DROP TABLE IF EXISTS test.destination; + ''') + + assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + # 4 tables, 2 consumers for each table => 8 consumer tags + assert int(result2) == 8 + + if __name__ == '__main__': cluster.start() raw_input("Cluster created, press any key to destroy...") From 1213161cf4201c201112cba5ac8bece9c0e6fd5e Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 6 Aug 2020 20:34:13 +0000 Subject: [PATCH 0033/1911] Add some message properties --- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 8 +-- src/Storages/RabbitMQ/StorageRabbitMQ.h | 3 +- .../WriteBufferToRabbitMQProducer.cpp | 66 ++++++++++++------- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 14 ++-- 4 files changed, 54 insertions(+), 37 deletions(-) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 80f66c6be0d..29a56934441 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -503,15 +503,11 @@ ConsumerBufferPtr StorageRabbitMQ::popReadBuffer(std::chrono::milliseconds timeo ConsumerBufferPtr StorageRabbitMQ::createReadBuffer() { - if (update_channel_id) - next_channel_id += num_queues; - update_channel_id = true; - ChannelPtr consumer_channel = std::make_shared(connection.get()); return std::make_shared( consumer_channel, setup_channel, event_handler, consumer_exchange, - next_channel_id, queue_base, log, row_delimiter, hash_exchange, num_queues, + ++consumer_id, queue_base, log, row_delimiter, hash_exchange, num_queues, deadletter_exchange, stream_cancelled); } @@ -520,7 +516,7 @@ ProducerBufferPtr StorageRabbitMQ::createWriteBuffer() { return std::make_shared( parsed_address, global_context, login_password, routing_keys, exchange_name, exchange_type, - log, use_transactional_channel, persistent, + ++producer_id, use_transactional_channel, persistent, log, row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 5aa030d821c..8e62305fd03 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -113,8 +113,7 @@ private: String sharding_exchange, bridge_exchange, consumer_exchange; std::once_flag flag; - size_t next_channel_id = 1; /// Must >= 1 because it is used as a binding key, which has to be > 0 - bool update_channel_id = false; + size_t producer_id = 0, consumer_id = 0; std::atomic loop_started = false, exchange_removed = false; ChannelPtr setup_channel; std::mutex connection_mutex, restore_connection; diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index c2ab8e3e843..429ca960378 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -24,9 +24,10 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( const Names & routing_keys_, const String & exchange_name_, const AMQP::ExchangeType exchange_type_, - Poco::Logger * log_, - const bool use_transactional_channel_, + const size_t channel_id_, + const bool use_tx_, const bool persistent_, + Poco::Logger * log_, std::optional delimiter, size_t rows_per_message, size_t chunk_size_) @@ -36,7 +37,8 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( , routing_keys(routing_keys_) , exchange_name(exchange_name_) , exchange_type(exchange_type_) - , use_transactional_channel(use_transactional_channel_) + , channel_id(std::to_string(channel_id_)) + , use_tx(use_tx_) , persistent(persistent_) , payloads(BATCH) , returned(BATCH << 6) @@ -50,7 +52,9 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( uv_loop_init(loop.get()); event_handler = std::make_unique(loop.get(), log); - /// New coonection for each publisher because cannot publish from different threads with the same connection.(https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) + /* New coonection for each publisher because cannot publish from different threads with the same connection. + * (See https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) + */ if (setupConnection()) setupChannel(); @@ -99,15 +103,17 @@ void WriteBufferToRabbitMQProducer::countRow() chunks.clear(); set(nullptr, 0); - payloads.push(payload); ++payload_counter; + payloads.push(std::make_pair(payload_counter, payload)); } } bool WriteBufferToRabbitMQProducer::setupConnection() { - connection = std::make_unique(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); + /// Need to manually restore connection if it is lost. + connection = std::make_unique(event_handler.get(), + AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); LOG_TRACE(log, "Trying to set up connection"); size_t cnt_retries = 0; @@ -118,10 +124,7 @@ bool WriteBufferToRabbitMQProducer::setupConnection() std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); } - if (!connection->ready()) - return false; - - return true; + return connection->ready(); } @@ -133,9 +136,14 @@ void WriteBufferToRabbitMQProducer::setupChannel() { LOG_ERROR(log, "Producer error: {}", message); - /// Means channel ends up in an error state and is not usable anymore. + /* Means channel ends up in an error state and is not usable anymore. + * (See https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/36#issuecomment-125112236) + */ producer_channel->close(); + if (use_tx) + return; + for (auto record = delivery_record.begin(); record != delivery_record.end(); record++) returned.tryPush(record->second); @@ -151,7 +159,7 @@ void WriteBufferToRabbitMQProducer::setupChannel() { LOG_DEBUG(log, "Producer channel is ready"); - if (use_transactional_channel) + if (use_tx) { producer_channel->startTransaction(); } @@ -211,24 +219,31 @@ void WriteBufferToRabbitMQProducer::removeConfirmed(UInt64 received_delivery_tag } -void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue & messages, bool republishing) +void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue> & messages, bool republishing) { - String payload; + std::pair payload; while (!messages.empty() && producer_channel->usable()) { messages.pop(payload); - AMQP::Envelope envelope(payload.data(), payload.size()); + AMQP::Envelope envelope(payload.second.data(), payload.second.size()); + + /// if headers exchange - routing keys are added here via headers, else - it is just empty. AMQP::Table message_settings = key_arguments; /* There is the case when connection is lost in the period after some messages were published and before ack/nack was sent by the * server, then it means that publisher will never now whether those messages were delivered or not, and therefore those records - * that received no ack/nack before connection loss will be republished, so there might be duplicates. To let consumer know that - * received message might be a possible duplicate - a "republished" field is added to message metadata. + * that received no ack/nack before connection loss will be republished (see onError() callback), so there might be duplicates. To + * let consumer know that received message might be a possible duplicate - a "republished" field is added to message metadata. */ message_settings["republished"] = std::to_string(republishing); envelope.setHeaders(message_settings); + /* Adding here a message_id property to message metadata. + * (See https://stackoverflow.com/questions/59384305/rabbitmq-how-to-handle-unwanted-duplicate-un-ack-message-after-connection-lost) + */ + envelope.setMessageID(channel_id + "-" + std::to_string(payload.first)); + /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse. if (persistent) envelope.setDeliveryMode(2); @@ -249,7 +264,7 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue & mes ++delivery_tag; delivery_record.insert(delivery_record.end(), {delivery_tag, payload}); - /// Need to break to let event loop run, because no publishing actually happend before looping. + /// Need to break at some point to let event loop run, because no publishing actually happend before looping. if (delivery_tag % BATCH == 0) break; } @@ -270,12 +285,14 @@ void WriteBufferToRabbitMQProducer::writingFunc() publish(returned, true); else if (!payloads.empty() && producer_channel->usable()) publish(payloads, false); + else if (use_tx) + break; iterateEventLoop(); if (wait_num.load() && delivery_record.empty() && payloads.empty() && returned.empty()) - wait_all.store(false); - else if ((!producer_channel->usable() && connection->usable()) || (!connection->usable() && setupConnection())) + wait_all = false; + else if ((!producer_channel->usable() && connection->usable()) || (!use_tx && !connection->usable() && setupConnection())) setupChannel(); } @@ -285,7 +302,12 @@ void WriteBufferToRabbitMQProducer::writingFunc() void WriteBufferToRabbitMQProducer::commit() { - if (!use_transactional_channel) + /* Actually have not yet found any information about how is it supposed work once any error occurs with a channel, because any channel + * error closes this channel and any operation on a closed channel will fail (but transaction is unique to channel). + * RabbitMQ transactions seem not trust-worthy at all - see https://www.rabbitmq.com/semantics.html. Seems like its best to always + * use "publisher confirms" rather than transactions (and by default it is so). Probably even need to delete this option. + */ + if (!use_tx || !producer_channel->usable()) return; std::atomic answer_received = false, wait_rollback = false; @@ -293,13 +315,11 @@ void WriteBufferToRabbitMQProducer::commit() .onSuccess([&]() { answer_received = true; - wait_all.store(false); LOG_TRACE(log, "All messages were successfully published"); }) .onError([&](const char * message1) { answer_received = true; - wait_all.store(false); LOG_TRACE(log, "Publishing not successful: {}", message1); wait_rollback = true; diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index b9378695d8d..0773863c31a 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -24,9 +24,10 @@ public: const Names & routing_keys_, const String & exchange_name_, const AMQP::ExchangeType exchange_type_, - Poco::Logger * log_, - const bool use_transactional_channel_, + const size_t channel_id_, + const bool use_tx_, const bool persistent_, + Poco::Logger * log_, std::optional delimiter, size_t rows_per_message, size_t chunk_size_ @@ -46,14 +47,15 @@ private: bool setupConnection(); void setupChannel(); void removeConfirmed(UInt64 received_delivery_tag, bool multiple, bool republish); - void publish(ConcurrentBoundedQueue & message, bool republishing); + void publish(ConcurrentBoundedQueue> & message, bool republishing); std::pair parsed_address; const std::pair login_password; const Names routing_keys; const String exchange_name; AMQP::ExchangeType exchange_type; - const bool use_transactional_channel; + const String channel_id; + const bool use_tx; const bool persistent; AMQP::Table key_arguments; @@ -64,12 +66,12 @@ private: std::unique_ptr connection; std::unique_ptr producer_channel; - ConcurrentBoundedQueue payloads, returned; + ConcurrentBoundedQueue> payloads, returned; UInt64 delivery_tag = 0; std::atomic wait_all = true; std::atomic wait_num = 0; UInt64 payload_counter = 0; - std::map delivery_record; + std::map> delivery_record; Poco::Logger * log; const std::optional delim; From 70fca95a5a63e1a9ddcdab60108ff330bbbf9f16 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 7 Aug 2020 11:42:04 +0300 Subject: [PATCH 0034/1911] mysql/postgresql: move Dockerfiles and docker_compose to docker/test --- .../integration/mysql_golang_client}/0.reference | 0 .../integration/mysql_golang_client}/Dockerfile | 3 +++ .../test/integration/mysql_golang_client}/main.go | 0 .../test/integration/mysql_java_client}/0.reference | 0 .../test/integration/mysql_java_client}/Dockerfile | 3 +++ .../test/integration/mysql_java_client}/Test.java | 0 docker/test/integration/mysql_js_client/Dockerfile | 8 ++++++++ .../test/integration/mysql_js_client}/test.js | 0 .../test/integration/mysql_php_client}/Dockerfile | 3 +++ .../test/integration/mysql_php_client}/client.crt | 0 .../test/integration/mysql_php_client}/client.key | 0 .../test/integration/mysql_php_client}/test.php | 0 .../test/integration/mysql_php_client}/test_ssl.php | 0 .../integration/postgresql_java_client}/0.reference | 0 .../integration/postgresql_java_client}/Dockerfile | 3 +++ .../integration/postgresql_java_client}/Test.java | 0 .../runner/compose/docker_compose_mysql_client.yml | 0 .../compose/docker_compose_mysql_golang_client.yml | 4 +--- .../compose/docker_compose_mysql_java_client.yml | 4 +--- .../compose/docker_compose_mysql_js_client.yml | 4 +--- .../compose/docker_compose_mysql_php_client.yml | 3 +-- .../runner/compose/docker_compose_postgesql.yml | 0 .../docker_compose_postgesql_java_client.yml | 4 +--- .../test_mysql_protocol/clients/mysqljs/Dockerfile | 5 ----- tests/integration/test_mysql_protocol/test.py | 13 +++++++------ tests/integration/test_postgresql_protocol/test.py | 5 +++-- 26 files changed, 35 insertions(+), 27 deletions(-) rename {tests/integration/test_mysql_protocol/clients/golang => docker/test/integration/mysql_golang_client}/0.reference (100%) rename {tests/integration/test_mysql_protocol/clients/golang => docker/test/integration/mysql_golang_client}/Dockerfile (52%) rename {tests/integration/test_mysql_protocol/clients/golang => docker/test/integration/mysql_golang_client}/main.go (100%) rename {tests/integration/test_mysql_protocol/clients/java => docker/test/integration/mysql_java_client}/0.reference (100%) rename {tests/integration/test_mysql_protocol/clients/java => docker/test/integration/mysql_java_client}/Dockerfile (84%) rename {tests/integration/test_mysql_protocol/clients/java => docker/test/integration/mysql_java_client}/Test.java (100%) create mode 100644 docker/test/integration/mysql_js_client/Dockerfile rename {tests/integration/test_mysql_protocol/clients/mysqljs => docker/test/integration/mysql_js_client}/test.js (100%) rename {tests/integration/test_mysql_protocol/clients/php-mysqlnd => docker/test/integration/mysql_php_client}/Dockerfile (65%) rename {tests/integration/test_mysql_protocol/clients/php-mysqlnd => docker/test/integration/mysql_php_client}/client.crt (100%) rename {tests/integration/test_mysql_protocol/clients/php-mysqlnd => docker/test/integration/mysql_php_client}/client.key (100%) rename {tests/integration/test_mysql_protocol/clients/php-mysqlnd => docker/test/integration/mysql_php_client}/test.php (100%) rename {tests/integration/test_mysql_protocol/clients/php-mysqlnd => docker/test/integration/mysql_php_client}/test_ssl.php (100%) rename {tests/integration/test_postgresql_protocol/clients/java => docker/test/integration/postgresql_java_client}/0.reference (100%) rename {tests/integration/test_postgresql_protocol/clients/java => docker/test/integration/postgresql_java_client}/Dockerfile (82%) rename {tests/integration/test_postgresql_protocol/clients/java => docker/test/integration/postgresql_java_client}/Test.java (100%) rename tests/integration/test_mysql_protocol/clients/mysql/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_client.yml (100%) rename tests/integration/test_mysql_protocol/clients/golang/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml (66%) rename tests/integration/test_mysql_protocol/clients/java/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml (65%) rename tests/integration/test_mysql_protocol/clients/mysqljs/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml (66%) rename tests/integration/test_mysql_protocol/clients/php-mysqlnd/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml (66%) rename tests/integration/test_postgresql_protocol/clients/psql/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_postgesql.yml (100%) rename tests/integration/test_postgresql_protocol/clients/java/docker_compose.yml => docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml (64%) delete mode 100644 tests/integration/test_mysql_protocol/clients/mysqljs/Dockerfile diff --git a/tests/integration/test_mysql_protocol/clients/golang/0.reference b/docker/test/integration/mysql_golang_client/0.reference similarity index 100% rename from tests/integration/test_mysql_protocol/clients/golang/0.reference rename to docker/test/integration/mysql_golang_client/0.reference diff --git a/tests/integration/test_mysql_protocol/clients/golang/Dockerfile b/docker/test/integration/mysql_golang_client/Dockerfile similarity index 52% rename from tests/integration/test_mysql_protocol/clients/golang/Dockerfile rename to docker/test/integration/mysql_golang_client/Dockerfile index d169c274a8b..4380383d1fb 100644 --- a/tests/integration/test_mysql_protocol/clients/golang/Dockerfile +++ b/docker/test/integration/mysql_golang_client/Dockerfile @@ -1,3 +1,6 @@ +# docker build -t yandex/clickhouse-mysql-golang-client . +# MySQL golang client docker container + FROM golang:1.12.2 RUN go get "github.com/go-sql-driver/mysql" diff --git a/tests/integration/test_mysql_protocol/clients/golang/main.go b/docker/test/integration/mysql_golang_client/main.go similarity index 100% rename from tests/integration/test_mysql_protocol/clients/golang/main.go rename to docker/test/integration/mysql_golang_client/main.go diff --git a/tests/integration/test_mysql_protocol/clients/java/0.reference b/docker/test/integration/mysql_java_client/0.reference similarity index 100% rename from tests/integration/test_mysql_protocol/clients/java/0.reference rename to docker/test/integration/mysql_java_client/0.reference diff --git a/tests/integration/test_mysql_protocol/clients/java/Dockerfile b/docker/test/integration/mysql_java_client/Dockerfile similarity index 84% rename from tests/integration/test_mysql_protocol/clients/java/Dockerfile rename to docker/test/integration/mysql_java_client/Dockerfile index 96713a68e66..fcb6a39f33b 100644 --- a/tests/integration/test_mysql_protocol/clients/java/Dockerfile +++ b/docker/test/integration/mysql_java_client/Dockerfile @@ -1,3 +1,6 @@ +# docker build -t yandex/clickhouse-mysql-java-client . +# MySQL Java client docker container + FROM ubuntu:18.04 RUN apt-get update && \ diff --git a/tests/integration/test_mysql_protocol/clients/java/Test.java b/docker/test/integration/mysql_java_client/Test.java similarity index 100% rename from tests/integration/test_mysql_protocol/clients/java/Test.java rename to docker/test/integration/mysql_java_client/Test.java diff --git a/docker/test/integration/mysql_js_client/Dockerfile b/docker/test/integration/mysql_js_client/Dockerfile new file mode 100644 index 00000000000..4f12de004ac --- /dev/null +++ b/docker/test/integration/mysql_js_client/Dockerfile @@ -0,0 +1,8 @@ +# docker build -t yandex/clickhouse-mysql-js-client . +# MySQL JavaScript client docker container + +FROM node:8 + +RUN npm install mysql + +COPY ./test.js test.js diff --git a/tests/integration/test_mysql_protocol/clients/mysqljs/test.js b/docker/test/integration/mysql_js_client/test.js similarity index 100% rename from tests/integration/test_mysql_protocol/clients/mysqljs/test.js rename to docker/test/integration/mysql_js_client/test.js diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/Dockerfile b/docker/test/integration/mysql_php_client/Dockerfile similarity index 65% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/Dockerfile rename to docker/test/integration/mysql_php_client/Dockerfile index 76125702076..e2ceb62f44f 100644 --- a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/Dockerfile +++ b/docker/test/integration/mysql_php_client/Dockerfile @@ -1,3 +1,6 @@ +# docker build -t yandex/clickhouse-mysql-php-client . +# MySQL PHP client docker container + FROM php:7.3-cli COPY ./client.crt client.crt diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/client.crt b/docker/test/integration/mysql_php_client/client.crt similarity index 100% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/client.crt rename to docker/test/integration/mysql_php_client/client.crt diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/client.key b/docker/test/integration/mysql_php_client/client.key similarity index 100% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/client.key rename to docker/test/integration/mysql_php_client/client.key diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/test.php b/docker/test/integration/mysql_php_client/test.php similarity index 100% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/test.php rename to docker/test/integration/mysql_php_client/test.php diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/test_ssl.php b/docker/test/integration/mysql_php_client/test_ssl.php similarity index 100% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/test_ssl.php rename to docker/test/integration/mysql_php_client/test_ssl.php diff --git a/tests/integration/test_postgresql_protocol/clients/java/0.reference b/docker/test/integration/postgresql_java_client/0.reference similarity index 100% rename from tests/integration/test_postgresql_protocol/clients/java/0.reference rename to docker/test/integration/postgresql_java_client/0.reference diff --git a/tests/integration/test_postgresql_protocol/clients/java/Dockerfile b/docker/test/integration/postgresql_java_client/Dockerfile similarity index 82% rename from tests/integration/test_postgresql_protocol/clients/java/Dockerfile rename to docker/test/integration/postgresql_java_client/Dockerfile index f08470ee805..eab236c9590 100644 --- a/tests/integration/test_postgresql_protocol/clients/java/Dockerfile +++ b/docker/test/integration/postgresql_java_client/Dockerfile @@ -1,3 +1,6 @@ +# docker build -t yandex/clickhouse-postgresql-java-client . +# PostgreSQL Java client docker container + FROM ubuntu:18.04 RUN apt-get update && \ diff --git a/tests/integration/test_postgresql_protocol/clients/java/Test.java b/docker/test/integration/postgresql_java_client/Test.java similarity index 100% rename from tests/integration/test_postgresql_protocol/clients/java/Test.java rename to docker/test/integration/postgresql_java_client/Test.java diff --git a/tests/integration/test_mysql_protocol/clients/mysql/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_client.yml similarity index 100% rename from tests/integration/test_mysql_protocol/clients/mysql/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_client.yml diff --git a/tests/integration/test_mysql_protocol/clients/golang/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml similarity index 66% rename from tests/integration/test_mysql_protocol/clients/golang/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml index 4fe6fdaeecd..34c39caa795 100644 --- a/tests/integration/test_mysql_protocol/clients/golang/docker_compose.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml @@ -1,8 +1,6 @@ version: '2.3' services: golang1: - build: - context: ./ - network: host + image: yandex/clickhouse-mysql-golang-client # to keep container running command: sleep infinity diff --git a/tests/integration/test_mysql_protocol/clients/java/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml similarity index 65% rename from tests/integration/test_mysql_protocol/clients/java/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml index 522f404cde6..9a556ce5a8e 100644 --- a/tests/integration/test_mysql_protocol/clients/java/docker_compose.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml @@ -1,8 +1,6 @@ version: '2.3' services: java1: - build: - context: ./ - network: host + image: yandex/clickhouse-mysql-java-client # to keep container running command: sleep infinity diff --git a/tests/integration/test_mysql_protocol/clients/mysqljs/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml similarity index 66% rename from tests/integration/test_mysql_protocol/clients/mysqljs/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml index ebb73bc611a..11645097354 100644 --- a/tests/integration/test_mysql_protocol/clients/mysqljs/docker_compose.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml @@ -1,8 +1,6 @@ version: '2.3' services: mysqljs1: - build: - context: ./ - network: host + image: yandex/clickhouse-mysql-js-client # to keep container running command: sleep infinity diff --git a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml similarity index 66% rename from tests/integration/test_mysql_protocol/clients/php-mysqlnd/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml index c197944f375..4a0616ca2fd 100644 --- a/tests/integration/test_mysql_protocol/clients/php-mysqlnd/docker_compose.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml @@ -1,7 +1,6 @@ version: '2.3' services: php1: - build: - context: ./ + image: yandex/clickhouse-mysql-php-client # to keep container running command: sleep infinity diff --git a/tests/integration/test_postgresql_protocol/clients/psql/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_postgesql.yml similarity index 100% rename from tests/integration/test_postgresql_protocol/clients/psql/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_postgesql.yml diff --git a/tests/integration/test_postgresql_protocol/clients/java/docker_compose.yml b/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml similarity index 64% rename from tests/integration/test_postgresql_protocol/clients/java/docker_compose.yml rename to docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml index 7094c8b2359..1b716dc514a 100644 --- a/tests/integration/test_postgresql_protocol/clients/java/docker_compose.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml @@ -1,8 +1,6 @@ version: '2.2' services: java: - build: - context: ./ - network: host + image: yandex/clickhouse-postgresql-java-client # to keep container running command: sleep infinity diff --git a/tests/integration/test_mysql_protocol/clients/mysqljs/Dockerfile b/tests/integration/test_mysql_protocol/clients/mysqljs/Dockerfile deleted file mode 100644 index 5381915efba..00000000000 --- a/tests/integration/test_mysql_protocol/clients/mysqljs/Dockerfile +++ /dev/null @@ -1,5 +0,0 @@ -FROM node:8 - -RUN npm install mysql - -COPY ./test.js test.js diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 4640c7b6b90..342fd5b451a 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -11,10 +11,11 @@ import pymysql.connections from docker.models.containers import Container -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, get_docker_compose_path SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DOCKER_COMPOSE_PATH = get_docker_compose_path() config_dir = os.path.join(SCRIPT_DIR, './configs') cluster = ClickHouseCluster(__file__) @@ -34,7 +35,7 @@ def server_address(): @pytest.fixture(scope='module') def mysql_client(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'mysql', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_mysql1_1') @@ -60,28 +61,28 @@ def mysql_server(mysql_client): @pytest.fixture(scope='module') def golang_container(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'golang', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_golang_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_golang1_1') @pytest.fixture(scope='module') def php_container(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'php-mysqlnd', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_php1_1') @pytest.fixture(scope='module') def nodejs_container(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'mysqljs', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_mysqljs1_1') @pytest.fixture(scope='module') def java_container(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'java', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_java1_1') diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index d9e2dfe3228..9a7d91b8fa2 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -20,6 +20,7 @@ psycopg2.extras.register_uuid() SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) config_dir = os.path.join(SCRIPT_DIR, './configs') +DOCKER_COMPOSE_PATH = get_docker_compose_path() cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node', config_dir=config_dir, env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) @@ -38,7 +39,7 @@ def server_address(): @pytest.fixture(scope='module') def psql_client(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'psql', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgesql.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_psql_1') @@ -61,7 +62,7 @@ def psql_server(psql_client): @pytest.fixture(scope='module') def java_container(): - docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'java', 'docker_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgesql_java_client.yml') subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_java_1') From 5d8acc3b1e6e791d6e8ec35e789b7d0af5eb1cd8 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 7 Aug 2020 11:47:56 +0300 Subject: [PATCH 0035/1911] Build client containers in CI --- docker/images.json | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/docker/images.json b/docker/images.json index 09114cc9710..0c4307f8e15 100644 --- a/docker/images.json +++ b/docker/images.json @@ -103,5 +103,25 @@ "docker/test/integration/helper_container": { "name": "yandex/clickhouse-integration-helper", "dependent": [] + }, + "docker/test/integration/mysql_golang_client": { + "name": "yandex/clickhouse-mysql-golang-client", + "dependent": [] + }, + "docker/test/integration/mysql_java_client": { + "name": "yandex/clickhouse-mysql-java-client", + "dependent": [] + }, + "docker/test/integration/mysql_js_client": { + "name": "yandex/clickhouse-mysql-js-client", + "dependent": [] + }, + "docker/test/integration/mysql_php_client": { + "name": "yandex/clickhouse-mysql-php-client", + "dependent": [] + }, + "docker/test/integration/postgresql_java_client": { + "name": "yandex/clickhouse-postgresql-java-client", + "dependent": [] } } From 222b06f4e9e6684db54e44abd3b7629f21a435d7 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 7 Aug 2020 13:42:39 +0300 Subject: [PATCH 0036/1911] Add lost import --- tests/integration/test_postgresql_protocol/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index 9a7d91b8fa2..47edafe757e 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -14,7 +14,7 @@ import subprocess import time import uuid -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, get_docker_compose_path psycopg2.extras.register_uuid() From eff0233184491ae96fffe087b5b85afb3fe6be09 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 6 Aug 2020 20:52:26 +0000 Subject: [PATCH 0037/1911] Update docs --- .../engines/table-engines/integrations/rabbitmq.md | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index e870471b4eb..41429016898 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -40,13 +40,13 @@ Required parameters: Optional parameters: -- `rabbitmq_exchange_type` – The type of RabbitMQ exchange: `direct`, `fanout`, `topic`, `headers`, `consistent-hash`. Default: `fanout`. +- `rabbitmq_exchange_type` – The type of RabbitMQ exchange: `direct`, `fanout`, `topic`, `headers`, `consistent_hash`. Default: `fanout`. - `rabbitmq_routing_key_list` – A comma-separated list of routing keys. - `rabbitmq_row_delimiter` – Delimiter character, which ends the message. - `rabbitmq_num_consumers` – The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. - `rabbitmq_num_queues` – The number of queues per consumer. Default: `1`. Specify more queues if the capacity of one queue per consumer is insufficient. - `rabbitmq_transactional_channel` – Wrap insert queries in transactions. Default: `0`. -- `rabbitmq_queue_base` - Specify a base name for queues that will be declared. This settings should be used to be able to restore reading from declared durable queues in case of some failure when not all messages were successfully consumed. Note: it makes sence only if messages are sent with delivery mode 2 (marked 'persistent', durable). To be able to resume consumption from one specific queue in case of failure - set its name in `rabbitmq_queue_base` setting and do not specify `rabbitmq_num_consumers` and `rabbitmq_num_queues` (defaults to 1). To be able to resume consumption from all queues, which were declared for a specific table - just specify the same settings: `rabbitmq_queue_base`, `rabbitmq_num_consumers`, `rabbitmq_num_queues`. +- `rabbitmq_queue_base` - Specify a base name for queues that will be declared. - `rabbitmq_deadletter_exchange` - Specify name for a [dead letter exchange](https://www.rabbitmq.com/dlx.html). You can create another table with this exchange name and collect messages in cases when they are republished to dead letter exchange. By default dead letter exchange is not specified. - `persistent` - If set to 1 (true), in insert query delivery mode will be set to 2 (marks messages as 'persistent'). Default: `0`. @@ -95,11 +95,18 @@ Exchange type options: - `headers` - Routing is based on `key=value` matches with a setting `x-match=all` or `x-match=any`. Example table key list: `x-match=all,format=logs,type=report,year=2020`. - `consistent-hash` - Data is evenly distributed between all bound tables (where exchange name is the same). Note that this exchange type must be enabled with RabbitMQ plugin: `rabbitmq-plugins enable rabbitmq_consistent_hash_exchange`. +Setting `rabbitmq_queue_base` may be used for the following cases: +- to be able to restore reading from certain durable queues when not all messages were successfully consumed. Note: it makes sence only if messages are sent with delivery mode 2 - marked 'persistent', durable. To be able to resume consumption from one specific queue - set its name in `rabbitmq_queue_base` setting and do not specify `rabbitmq_num_consumers` and `rabbitmq_num_queues` (defaults to 1). To be able to resume consumption from all queues, which were declared for a specific table - just specify the same settings: `rabbitmq_queue_base`, `rabbitmq_num_consumers`, `rabbitmq_num_queues`. By default, queue names will be unique to tables. +- to reuse queues as they are declared durable and not auto-deleted. +- to let different tables share queues, so that multiple consumers could be registered for the same queues, which makes better performance. If using `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` settings, the exact match of queues is achieved in case these parameters are the same. + If `rabbitmq_num_consumers` and/or `rabbitmq_num_queues` settings are specified along with `rabbitmq_exchange_type`, then: - `rabbitmq-consistent-hash-exchange` plugin must be enabled. - `message_id` property of the published messages must be specified (unique for each message/batch). +For insert query there is message metadata, which is added for each published message: messageID and republished flag - can be accessed via message headers. + Do not use the same table for inserts and materialized views. Example: @@ -116,7 +123,7 @@ Example: rabbitmq_num_consumers = 5; CREATE TABLE daily (key UInt64, value UInt64) - ENGINE = MergeTree(); + ENGINE = MergeTree() ORDER BY key; CREATE MATERIALIZED VIEW consumer TO daily AS SELECT key, value FROM queue; From 2ea32a710a0ba12ff533b3b4cf083890ccd7e136 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 8 Aug 2020 16:45:52 +0000 Subject: [PATCH 0038/1911] More tests, better reconnect --- src/Storages/RabbitMQ/RabbitMQHandler.cpp | 11 +- src/Storages/RabbitMQ/RabbitMQHandler.h | 12 +- .../ReadBufferFromRabbitMQConsumer.cpp | 6 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 46 +++-- .../WriteBufferToRabbitMQProducer.cpp | 113 ++++++++---- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 3 +- .../integration/test_storage_rabbitmq/test.py | 168 +++++++++++++++++- 7 files changed, 295 insertions(+), 64 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.cpp b/src/Storages/RabbitMQ/RabbitMQHandler.cpp index c7186e3d3ff..d6b6ab440b2 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.cpp +++ b/src/Storages/RabbitMQ/RabbitMQHandler.cpp @@ -11,7 +11,9 @@ namespace DB RabbitMQHandler::RabbitMQHandler(uv_loop_t * loop_, Poco::Logger * log_) : AMQP::LibUvHandler(loop_), loop(loop_), - log(log_) + log(log_), + connection_running(false), + loop_state(Loop::STOP) { } @@ -27,15 +29,16 @@ void RabbitMQHandler::onError(AMQP::TcpConnection * connection, const char * mes void RabbitMQHandler::onReady(AMQP::TcpConnection * /* connection */) { - LOG_TRACE(log, "Connection is ready"); connection_running.store(true); + LOG_TRACE(log, "Connection is ready"); + + loop_state.store(Loop::RUN); } void RabbitMQHandler::startLoop() { std::lock_guard lock(startup_mutex); - /// stop_loop variable is updated in a separate thread - while (!stop_loop.load() && connection_running.load()) + while (loop_state.load() == Loop::RUN) uv_run(loop, UV_RUN_NOWAIT); } diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.h b/src/Storages/RabbitMQ/RabbitMQHandler.h index 2a992f68d27..3c0c5a2af37 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.h +++ b/src/Storages/RabbitMQ/RabbitMQHandler.h @@ -11,6 +11,12 @@ namespace DB { +namespace Loop +{ + static const UInt8 RUN = 1; + static const UInt8 STOP = 2; +} + class RabbitMQHandler : public AMQP::LibUvHandler { @@ -19,16 +25,18 @@ public: void onError(AMQP::TcpConnection * connection, const char * message) override; void onReady(AMQP::TcpConnection * connection) override; - void stop() { stop_loop.store(true); } void startLoop(); void iterateLoop(); bool connectionRunning() { return connection_running.load(); } + void updateLoopState(UInt8 state) { loop_state.store(state); } + UInt8 getLoopState() { return loop_state.load(); } private: uv_loop_t * loop; Poco::Logger * log; - std::atomic stop_loop = false, connection_running = false; + std::atomic connection_running; + std::atomic loop_state; std::mutex startup_mutex; }; diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 47c15df3bd3..d12d08fad25 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -51,7 +51,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( { consumer_channel->onError([&](const char * message) { - LOG_ERROR(log, "Consumer {} error: {}", consumer_tag, message); + LOG_ERROR(log, "Consumer {} error: {}", channel_id, message); channel_error.store(true); }); @@ -129,7 +129,7 @@ void ReadBufferFromRabbitMQConsumer::subscribe() { if (consumer_tag.empty()) consumer_tag = consumer; - LOG_TRACE(log, "Consumer {} is subscribed to queue {}, consumer tag {}", channel_id, queue_name, consumer); + LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); }) .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) { @@ -157,7 +157,7 @@ void ReadBufferFromRabbitMQConsumer::ackMessages() { prev_tag = delivery_tag; consumer_channel->ack(prev_tag, AMQP::multiple); /// Will ack all up to last tag staring from last acked. - LOG_TRACE(log, "Consumer {} acknowledged messages with deliveryTags up to {}", consumer_tag, prev_tag); + LOG_TRACE(log, "Consumer {} acknowledged messages with deliveryTags up to {}", channel_id, prev_tag); } } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 29a56934441..f0b58d3e722 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -111,7 +111,7 @@ StorageRabbitMQ::StorageRabbitMQ( if (!connection->ready()) { uv_loop_close(loop.get()); - throw Exception("Cannot set up connection for consumers", ErrorCodes::CANNOT_CONNECT_RABBITMQ); + throw Exception("Cannot connect to RabbitMQ", ErrorCodes::CANNOT_CONNECT_RABBITMQ); } rabbitmq_context.makeQueryContext(); @@ -161,7 +161,7 @@ StorageRabbitMQ::StorageRabbitMQ( * at the same time sharding exchange is needed (if there are multiple shared queues), then those tables also need * to share sharding exchange. */ - sharding_exchange = exchange_name + queue_base; + sharding_exchange = exchange_name + "_" + queue_base; } bridge_exchange = sharding_exchange + "_bridge"; @@ -319,7 +319,7 @@ void StorageRabbitMQ::unbindExchange() event_handler->iterateLoop(); } - event_handler->stop(); + event_handler->updateLoopState(Loop::STOP); looping_task->deactivate(); heartbeat_task->deactivate(); }); @@ -335,31 +335,40 @@ bool StorageRabbitMQ::restoreConnection() if (!connection->usable() || !connection->ready()) { - LOG_TRACE(log, "Trying to restore consumer connection"); + if (event_handler->getLoopState() == Loop::RUN) + { + event_handler->updateLoopState(Loop::STOP); + looping_task->deactivate(); + heartbeat_task->deactivate(); + } + /* connection->close() is called in onError() method (called by the AMQP library when a fatal error occurs on the connection) + * inside event_handler, but it is not closed immediately (firstly, all pending operations are completed, and then an AMQP + * closing-handshake is performed). But cannot open a new connection untill previous one is properly closed). + */ + size_t cnt_retries = 0; + while (!connection->closed() && ++cnt_retries != (RETRIES_MAX >> 1)) + event_handler->iterateLoop(); + + /// This will force immediate closure if not yet closed. if (!connection->closed()) - connection->close(); + connection->close(true); + LOG_TRACE(log, "Trying to restore consumer connection"); connection = std::make_shared(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); - size_t cnt_retries = 0; + cnt_retries = 0; while (!connection->ready() && ++cnt_retries != RETRIES_MAX) { event_handler->iterateLoop(); std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); } - } - if (event_handler->connectionRunning()) - { - LOG_TRACE(log, "Connection restored"); - - heartbeat_task->scheduleAfter(HEARTBEAT_RESCHEDULE_MS); - looping_task->activateAndSchedule(); - } - else - { - LOG_TRACE(log, "Connection refused"); + if (event_handler->connectionRunning()) + { + looping_task->activateAndSchedule(); + heartbeat_task->scheduleAfter(HEARTBEAT_RESCHEDULE_MS); + } } restore_connection.unlock(); @@ -451,8 +460,7 @@ void StorageRabbitMQ::startup() void StorageRabbitMQ::shutdown() { stream_cancelled = true; - - event_handler->stop(); + event_handler->updateLoopState(Loop::STOP); looping_task->deactivate(); streaming_task->deactivate(); diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 429ca960378..78920bc13c6 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -52,10 +52,10 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( uv_loop_init(loop.get()); event_handler = std::make_unique(loop.get(), log); - /* New coonection for each publisher because cannot publish from different threads with the same connection. - * (See https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) + /* New coonection for each producer buffer because cannot publish from different threads with the same connection. + * (https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) */ - if (setupConnection()) + if (setupConnection(false)) setupChannel(); writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); @@ -103,21 +103,41 @@ void WriteBufferToRabbitMQProducer::countRow() chunks.clear(); set(nullptr, 0); - ++payload_counter; - payloads.push(std::make_pair(payload_counter, payload)); + if (!use_tx) + { + /// "publisher confirms" will be used, this is default. + ++payload_counter; + payloads.push(std::make_pair(payload_counter, payload)); + } + else + { + /// means channel->startTransaction() was called, not default, enabled only with table setting. + publish(payload); + } } } -bool WriteBufferToRabbitMQProducer::setupConnection() +bool WriteBufferToRabbitMQProducer::setupConnection(bool reconnecting) { - /// Need to manually restore connection if it is lost. + size_t cnt_retries = 0; + if (reconnecting) + { + /* connection->close() is called in onError() method (called by the AMQP library when a fatal error occurs on the connection) + * inside event_handler, but it is not closed immediately (firstly, all pending operations are completed, and then an AMQP + * closing-handshake is performed). But cannot open a new connection untill previous one is properly closed). + */ + while (!connection->closed() && ++cnt_retries != (RETRIES_MAX >> 1)) + event_handler->iterateLoop(); + if (!connection->closed()) + connection->close(true); + } + + LOG_TRACE(log, "Trying to set up connection"); connection = std::make_unique(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); - LOG_TRACE(log, "Trying to set up connection"); - size_t cnt_retries = 0; - + cnt_retries = 0; while (!connection->ready() && ++cnt_retries != RETRIES_MAX) { event_handler->iterateLoop(); @@ -136,16 +156,12 @@ void WriteBufferToRabbitMQProducer::setupChannel() { LOG_ERROR(log, "Producer error: {}", message); - /* Means channel ends up in an error state and is not usable anymore. - * (See https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/36#issuecomment-125112236) - */ + /// Channel is not usable anymore. (https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/36#issuecomment-125112236) producer_channel->close(); - if (use_tx) - return; - - for (auto record = delivery_record.begin(); record != delivery_record.end(); record++) - returned.tryPush(record->second); + /// Records that have not received ack/nack from server before channel closure. + for (const auto & record : delivery_record) + returned.tryPush(record.second); LOG_DEBUG(log, "Currently {} messages have not been confirmed yet, {} waiting to be published, {} will be republished", delivery_record.size(), payloads.size(), returned.size()); @@ -240,7 +256,7 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueueusable()) - publish(returned, true); - else if (!payloads.empty() && producer_channel->usable()) - publish(payloads, false); - else if (use_tx) - break; + /// This check is to make sure that delivery_record.size() is never bigger than returned.size() + if (delivery_record.size() < (BATCH << 6)) + { + /* Publish main paylods only when there are no returned messages. This way it is ensured that returned.queue never grows too + * big and returned messages are republished as fast as possible. Also payloads.queue is fixed size and push attemt would + * block thread in countRow() once there is no space - that is intended. + */ + if (!returned.empty() && producer_channel->usable()) + publish(returned, true); + else if (!payloads.empty() && producer_channel->usable()) + publish(payloads, false); + } iterateEventLoop(); if (wait_num.load() && delivery_record.empty() && payloads.empty() && returned.empty()) wait_all = false; - else if ((!producer_channel->usable() && connection->usable()) || (!use_tx && !connection->usable() && setupConnection())) + else if ((!producer_channel->usable() && connection->usable()) || (!connection->usable() && setupConnection(true))) setupChannel(); } @@ -300,9 +321,34 @@ void WriteBufferToRabbitMQProducer::writingFunc() } +/* This publish is for the case when transaction is delcared on the channel with channel->startTransaction(). Here only publish + * once payload is available and then commitTransaction() is called, where a needed event loop will run. + */ +void WriteBufferToRabbitMQProducer::publish(const String & payload) +{ + AMQP::Envelope envelope(payload.data(), payload.size()); + + if (persistent) + envelope.setDeliveryMode(2); + + if (exchange_type == AMQP::ExchangeType::consistent_hash) + { + producer_channel->publish(exchange_name, std::to_string(delivery_tag), envelope); + } + else if (exchange_type == AMQP::ExchangeType::headers) + { + producer_channel->publish(exchange_name, "", envelope); + } + else + { + producer_channel->publish(exchange_name, routing_keys[0], envelope); + } +} + + void WriteBufferToRabbitMQProducer::commit() { - /* Actually have not yet found any information about how is it supposed work once any error occurs with a channel, because any channel + /* Actually have not yet found any information about how is it supposed work once any error occurs with a channel, because any channel * error closes this channel and any operation on a closed channel will fail (but transaction is unique to channel). * RabbitMQ transactions seem not trust-worthy at all - see https://www.rabbitmq.com/semantics.html. Seems like its best to always * use "publisher confirms" rather than transactions (and by default it is so). Probably even need to delete this option. @@ -311,6 +357,7 @@ void WriteBufferToRabbitMQProducer::commit() return; std::atomic answer_received = false, wait_rollback = false; + producer_channel->commitTransaction() .onSuccess([&]() { @@ -320,9 +367,9 @@ void WriteBufferToRabbitMQProducer::commit() .onError([&](const char * message1) { answer_received = true; + wait_rollback = true; LOG_TRACE(log, "Publishing not successful: {}", message1); - wait_rollback = true; producer_channel->rollbackTransaction() .onSuccess([&]() { @@ -330,8 +377,8 @@ void WriteBufferToRabbitMQProducer::commit() }) .onError([&](const char * message2) { - LOG_ERROR(log, "Failed to rollback transaction: {}", message2); wait_rollback = false; + LOG_ERROR(log, "Failed to rollback transaction: {}", message2); }); }); diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 0773863c31a..95d505bafd5 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -44,10 +44,11 @@ private: void nextImpl() override; void iterateEventLoop(); void writingFunc(); - bool setupConnection(); + bool setupConnection(bool reconnecting); void setupChannel(); void removeConfirmed(UInt64 received_delivery_tag, bool multiple, bool republish); void publish(ConcurrentBoundedQueue> & message, bool republishing); + void publish(const String & payload); std::pair parsed_address; const std::pair login_password; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index be45298b52f..a670ea8ab54 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -86,6 +86,18 @@ def rabbitmq_check_result(result, check=False, ref_file='test_rabbitmq_json.refe return TSV(result) == TSV(reference) +def kill_rabbitmq(): + p = subprocess.Popen(('docker', 'stop', rabbitmq_id), stdout=subprocess.PIPE) + p.communicate() + return p.returncode == 0 + + +def revive_rabbitmq(): + p = subprocess.Popen(('docker', 'start', rabbitmq_id), stdout=subprocess.PIPE) + p.communicate() + return p.returncode == 0 + + # Fixtures @pytest.fixture(scope="module") @@ -1684,7 +1696,7 @@ def test_rabbitmq_queue_resume_2(rabbitmq_cluster): while True: result1 = instance.query('SELECT count() FROM test.view') time.sleep(1) - if int(result1) > collected: + if int(result1) == messages_num * threads_num: break instance.query(''' @@ -1693,7 +1705,7 @@ def test_rabbitmq_queue_resume_2(rabbitmq_cluster): DROP TABLE IF EXISTS test.view; ''') - assert int(result1) > collected, 'ClickHouse lost some messages: {}'.format(result) + assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) @pytest.mark.timeout(420) @@ -1868,6 +1880,158 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): assert int(result2) == 8 +@pytest.mark.timeout(420) +def test_rabbitmq_consumer_restore_connection(rabbitmq_cluster): + instance.query(''' + CREATE TABLE test.consumer_reconnect (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'consumer_reconnect', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + ''') + + i = [0] + messages_num = 5000 + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + def produce(): + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + messages = [] + for _ in range(messages_num): + messages.append(json.dumps({'key': i[0], 'value': i[0]})) + i[0] += 1 + for message in messages: + channel.basic_publish(exchange='consumer_reconnect', routing_key='', body=message, properties=pika.BasicProperties(delivery_mode = 2)) + connection.close() + + threads = [] + threads_num = 20 + for _ in range(threads_num): + threads.append(threading.Thread(target=produce)) + for thread in threads: + time.sleep(random.uniform(0, 1)) + thread.start() + + for thread in threads: + thread.join() + + instance.query(''' + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.consumer_reconnect; + ''') + + while int(instance.query('SELECT count() FROM test.view')) == 0: + time.sleep(1) + + kill_rabbitmq(); + time.sleep(4); + revive_rabbitmq(); + + collected = int(instance.query('SELECT count() FROM test.view')) + + while True: + result = instance.query('SELECT count() FROM test.view') + time.sleep(1) + print("receiived", result, "collected", collected) + if int(result) >= messages_num * threads_num: + break + + instance.query(''' + DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer_reconnect; + ''') + + # >= because at-least-once + assert int(result) >= messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + + +@pytest.mark.timeout(420) +def test_rabbitmq_producer_restore_connection(rabbitmq_cluster): + instance.query(''' + DROP TABLE IF EXISTS test.destination; + CREATE TABLE test.destination(key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + ''') + + instance.query(''' + DROP TABLE IF EXISTS test.consume; + DROP TABLE IF EXISTS test.consume_mv; + CREATE TABLE test.consume (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'producer_reconnect', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + CREATE MATERIALIZED VIEW test.consume_mv TO test.destination AS + SELECT key, value FROM test.consume; + ''') + + instance.query(''' + DROP TABLE IF EXISTS test.producer_reconnect; + CREATE TABLE test.producer_reconnect (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'producer_reconnect', + rabbitmq_persistent_mode = '1', + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + ''') + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + + messages_num = 100000 + values = [] + for i in range(messages_num): + values.append("({i}, {i})".format(i=i)) + values = ','.join(values) + + while True: + try: + instance.query("INSERT INTO test.producer_reconnect VALUES {}".format(values)) + break + except QueryRuntimeException as e: + if 'Local: Timed out.' in str(e): + continue + else: + raise + + while int(instance.query('SELECT count() FROM test.destination')) == 0: + time.sleep(0.1) + + kill_rabbitmq(); + time.sleep(4); + revive_rabbitmq(); + + while True: + result = instance.query('SELECT count() FROM test.destination') + time.sleep(1) + print(result, messages_num) + if int(result) >= messages_num: + break + + instance.query(''' + DROP TABLE IF EXISTS test.consume_mv; + DROP TABLE IF EXISTS test.consume; + DROP TABLE IF EXISTS test.producer_reconnect; + DROP TABLE IF EXISTS test.destination; + ''') + + assert int(result) >= messages_num, 'ClickHouse lost some messages: {}'.format(result) + + if __name__ == '__main__': cluster.start() raw_input("Cluster created, press any key to destroy...") From 1d0e4ca6706a9002bbbe92499699f3628c101128 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 10 Aug 2020 16:35:08 +0300 Subject: [PATCH 0039/1911] pass tag to docker_compose --- .../docker_compose_mysql_golang_client.yml | 2 +- .../docker_compose_mysql_java_client.yml | 2 +- .../docker_compose_mysql_js_client.yml | 2 +- .../docker_compose_mysql_php_client.yml | 2 +- .../docker_compose_postgesql_java_client.yml | 2 +- .../integration/runner/dockerd-entrypoint.sh | 6 +++++ tests/integration/runner | 24 ++++++++++++++++++- 7 files changed, 34 insertions(+), 6 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml b/docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml index 34c39caa795..b172cbcb2c6 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_golang_client.yml @@ -1,6 +1,6 @@ version: '2.3' services: golang1: - image: yandex/clickhouse-mysql-golang-client + image: yandex/clickhouse-mysql-golang-client:${DOCKER_MYSQL_GOLANG_CLIENT_TAG} # to keep container running command: sleep infinity diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml b/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml index 9a556ce5a8e..be1b3ad3f72 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml @@ -1,6 +1,6 @@ version: '2.3' services: java1: - image: yandex/clickhouse-mysql-java-client + image: yandex/clickhouse-mysql-java-client:${DOCKER_MYSQL_JAVA_CLIENT_TAG} # to keep container running command: sleep infinity diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml b/docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml index 11645097354..83954229111 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_js_client.yml @@ -1,6 +1,6 @@ version: '2.3' services: mysqljs1: - image: yandex/clickhouse-mysql-js-client + image: yandex/clickhouse-mysql-js-client:${DOCKER_MYSQL_JS_CLIENT_TAG} # to keep container running command: sleep infinity diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml b/docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml index 4a0616ca2fd..e61cb193b0e 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_php_client.yml @@ -1,6 +1,6 @@ version: '2.3' services: php1: - image: yandex/clickhouse-mysql-php-client + image: yandex/clickhouse-mysql-php-client:${DOCKER_MYSQL_PHP_CLIENT_TAG} # to keep container running command: sleep infinity diff --git a/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml b/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml index 1b716dc514a..ef18d1edd7b 100644 --- a/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml @@ -1,6 +1,6 @@ version: '2.2' services: java: - image: yandex/clickhouse-postgresql-java-client + image: yandex/clickhouse-postgresql-java-client:${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG} # to keep container running command: sleep infinity diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 8b0682396f8..6c2eaba6225 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -22,5 +22,11 @@ export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-config export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge +export ${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} +export ${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest} +export ${DOCKER_MYSQL_JS_CLIENT_TAG:=latest} +export ${DOCKER_MYSQL_PHP_CLIENT_TAG:=latest} +export ${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest} + cd /ClickHouse/tests/integration exec "$@" diff --git a/tests/integration/runner b/tests/integration/runner index 6369ebeea3e..ddd4c79b127 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -124,6 +124,10 @@ if __name__ == "__main__": default="latest", help="Version of docker image which runner will use to run tests") + parser.add_argument( + "--docker-compose-images-tags", + action="append", + help="Set non-default tags for images used in docker compose recipes(yandex/my_container:my_tag)") parser.add_argument('pytest_args', nargs='*', help="args for pytest command") @@ -135,6 +139,23 @@ if __name__ == "__main__": if not args.disable_net_host: net = "--net=host" + env_tags = "" + + for img_tag in args.docker_compose_images_tags: + [image, tag] = img_tag.split(":") + if image == "yandex/clickhouse-mysql-golang-client": + env_tags += "-e {}={}".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-mysql-java-client": + env_tags += "-e {}={}".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-mysql-js-client": + env_tags += "-e {}={}".format("DOCKER_MYSQL_JS_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-mysql-php-client": + env_tags += "-e {}={}".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-postgresql-java-client": + env_tags += "-e {}={}".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) + else: + raise Exception("Unknown image {}".format(image)) + # create named volume which will be used inside to store images and other docker related files, # to avoid redownloading it every time # @@ -148,13 +169,14 @@ if __name__ == "__main__": cmd = "docker run {net} {tty} --rm --name {name} --privileged --volume={bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ - --volume={name}_volume:/var/lib/docker -e PYTEST_OPTS='{opts}' {img} {command}".format( + --volume={name}_volume:/var/lib/docker {env_tags} -e PYTEST_OPTS='{opts}' {img} {command}".format( net=net, tty=tty, bin=args.binary, bridge_bin=args.bridge_binary, base_cfg=args.base_configs_dir, cases_dir=args.cases_dir, + env_tags=env_tags, opts=' '.join(args.pytest_args), img=DIND_INTEGRATION_TESTS_IMAGE_NAME + ":" + args.docker_image_version, name=CONTAINER_NAME, From dd2449354a567adca3bc36fb889e07d9260ee5fd Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 11 Aug 2020 11:28:16 +0300 Subject: [PATCH 0040/1911] fix --- tests/integration/runner | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index ddd4c79b127..66f6e2d797d 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -144,15 +144,15 @@ if __name__ == "__main__": for img_tag in args.docker_compose_images_tags: [image, tag] = img_tag.split(":") if image == "yandex/clickhouse-mysql-golang-client": - env_tags += "-e {}={}".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) + env_tags += "-e {}={} ".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) elif image == "yandex/clickhouse-mysql-java-client": - env_tags += "-e {}={}".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) + env_tags += "-e {}={} ".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) elif image == "yandex/clickhouse-mysql-js-client": - env_tags += "-e {}={}".format("DOCKER_MYSQL_JS_CLIENT_TAG", tag) + env_tags += "-e {}={} ".format("DOCKER_MYSQL_JS_CLIENT_TAG", tag) elif image == "yandex/clickhouse-mysql-php-client": - env_tags += "-e {}={}".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) + env_tags += "-e {}={} ".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) elif image == "yandex/clickhouse-postgresql-java-client": - env_tags += "-e {}={}".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) + env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) else: raise Exception("Unknown image {}".format(image)) From e3f7fea33a6098a5f4dee4af729cbaa4c37213fc Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 11 Aug 2020 11:31:26 +0300 Subject: [PATCH 0041/1911] fix --- docker/test/integration/runner/dockerd-entrypoint.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 6c2eaba6225..c38260279ed 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -22,11 +22,11 @@ export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-config export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge -export ${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} -export ${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest} -export ${DOCKER_MYSQL_JS_CLIENT_TAG:=latest} -export ${DOCKER_MYSQL_PHP_CLIENT_TAG:=latest} -export ${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest} +export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} +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} cd /ClickHouse/tests/integration exec "$@" From 5d52c306c8c40974e9d1819202c1c4f891924ce2 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 11 Aug 2020 11:35:31 +0300 Subject: [PATCH 0042/1911] fix --- tests/integration/runner | 29 +++++++++++++++-------------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index 66f6e2d797d..20737fa9a2f 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -141,20 +141,21 @@ if __name__ == "__main__": env_tags = "" - for img_tag in args.docker_compose_images_tags: - [image, tag] = img_tag.split(":") - if image == "yandex/clickhouse-mysql-golang-client": - env_tags += "-e {}={} ".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) - elif image == "yandex/clickhouse-mysql-java-client": - env_tags += "-e {}={} ".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) - elif image == "yandex/clickhouse-mysql-js-client": - env_tags += "-e {}={} ".format("DOCKER_MYSQL_JS_CLIENT_TAG", tag) - elif image == "yandex/clickhouse-mysql-php-client": - env_tags += "-e {}={} ".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) - elif image == "yandex/clickhouse-postgresql-java-client": - env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) - else: - raise Exception("Unknown image {}".format(image)) + if args.docker_compose_images_tags in not None: + for img_tag in args.docker_compose_images_tags: + [image, tag] = img_tag.split(":") + if image == "yandex/clickhouse-mysql-golang-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-mysql-java-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-mysql-js-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_JS_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-mysql-php-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-postgresql-java-client": + env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) + else: + raise Exception("Unknown image {}".format(image)) # create named volume which will be used inside to store images and other docker related files, # to avoid redownloading it every time From 75d0b8245fa207775236c443c725e6c949b30841 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 11 Aug 2020 11:36:26 +0300 Subject: [PATCH 0043/1911] fix typo --- tests/integration/runner | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/runner b/tests/integration/runner index 20737fa9a2f..e5d6eabe794 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -141,7 +141,7 @@ if __name__ == "__main__": env_tags = "" - if args.docker_compose_images_tags in not None: + if args.docker_compose_images_tags is not None: for img_tag in args.docker_compose_images_tags: [image, tag] = img_tag.split(":") if image == "yandex/clickhouse-mysql-golang-client": From fdd18e540dd0a15df81c4b1f3d75ac1aa6ad1d4a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Aug 2020 07:06:23 +0300 Subject: [PATCH 0044/1911] move reference files --- .../integration/test_mysql_protocol/golang.reference | 0 .../integration/test_mysql_protocol/java.reference | 0 .../integration/test_postgresql_protocol/java.reference | 0 3 files changed, 0 insertions(+), 0 deletions(-) rename docker/test/integration/mysql_golang_client/0.reference => tests/integration/test_mysql_protocol/golang.reference (100%) rename docker/test/integration/mysql_java_client/0.reference => tests/integration/test_mysql_protocol/java.reference (100%) rename docker/test/integration/postgresql_java_client/0.reference => tests/integration/test_postgresql_protocol/java.reference (100%) diff --git a/docker/test/integration/mysql_golang_client/0.reference b/tests/integration/test_mysql_protocol/golang.reference similarity index 100% rename from docker/test/integration/mysql_golang_client/0.reference rename to tests/integration/test_mysql_protocol/golang.reference diff --git a/docker/test/integration/mysql_java_client/0.reference b/tests/integration/test_mysql_protocol/java.reference similarity index 100% rename from docker/test/integration/mysql_java_client/0.reference rename to tests/integration/test_mysql_protocol/java.reference diff --git a/docker/test/integration/postgresql_java_client/0.reference b/tests/integration/test_postgresql_protocol/java.reference similarity index 100% rename from docker/test/integration/postgresql_java_client/0.reference rename to tests/integration/test_postgresql_protocol/java.reference From 9767d9627499faf4eb9dcb7a0a0d9575a12a1863 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Aug 2020 11:16:46 +0300 Subject: [PATCH 0045/1911] Fix names --- tests/integration/test_mysql_protocol/test.py | 4 ++-- tests/integration/test_postgresql_protocol/test.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 342fd5b451a..a31961dbd16 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -329,7 +329,7 @@ def test_python_client(server_address): def test_golang_client(server_address, golang_container): # type: (str, Container) -> None - with open(os.path.join(SCRIPT_DIR, 'clients', 'golang', '0.reference')) as fp: + with open(os.path.join(SCRIPT_DIR,'golang.reference')) as fp: reference = fp.read() code, (stdout, stderr) = golang_container.exec_run('./main --host {host} --port {port} --user default --password 123 --database ' @@ -386,7 +386,7 @@ def test_mysqljs_client(server_address, nodejs_container): def test_java_client(server_address, java_container): # type: (str, Container) -> None - with open(os.path.join(SCRIPT_DIR, 'clients', 'java', '0.reference')) as fp: + with open(os.path.join(SCRIPT_DIR, 'java.reference')) as fp: reference = fp.read() # database not exists exception. diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index 47edafe757e..527c652229e 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -133,7 +133,7 @@ def test_python_client(server_address): def test_java_client(server_address, java_container): - with open(os.path.join(SCRIPT_DIR, 'clients', 'java', '0.reference')) as fp: + with open(os.path.join(SCRIPT_DIR, 'java.reference')) as fp: reference = fp.read() # database not exists exception. From 160776f183116bfacac1d61e51c2821cb06bbcb8 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Aug 2020 11:55:04 +0300 Subject: [PATCH 0046/1911] Add explicit main_configs, user_configs and dictionaries in integration tests. --- .../integration/runner/dockerd-entrypoint.sh | 3 +- tests/integration/CMakeLists.txt | 2 +- .../helpers/0_common_enable_dictionaries.xml | 4 + tests/integration/helpers/client.py | 2 +- tests/integration/helpers/cluster.py | 165 +++++++++++------ tests/integration/helpers/dictonaries | 1 + tests/integration/helpers/test_tools.py | 3 + tests/integration/runner | 2 +- .../test_access_control_on_cluster/test.py | 6 +- .../test_adaptive_granularity/test.py | 24 +-- .../test_allowed_client_hosts/test.py | 10 +- .../test_allowed_url_from_config/test.py | 18 +- .../test_atomic_drop_table/test.py | 2 +- .../configs/conf.d/clusters.xml | 150 ++++++++-------- tests/integration/test_cluster_copier/test.py | 10 +- .../test_cluster_copier/trivial_test.py | 2 +- .../test_config_corresponding_root/test.py | 4 +- .../integration/test_custom_settings/test.py | 2 +- .../configs/disable_ssl_verification.xml | 12 ++ .../configs/enable_dictionaries.xml | 4 + .../test.py | 18 +- .../configs/enable_dictionaries.xml | 5 + .../test.py | 7 +- .../configs/allow_remote_node.xml | 5 + .../dictionaries/conflict_name_dictionary.xml | 41 +++++ .../dictionaries/lazy_load_dictionary.xml | 4 + .../configs/enable_dictionaries.xml | 3 + .../configs/user_admin.xml | 36 ++++ .../integration/test_dictionaries_ddl/test.py | 14 +- .../configs/enable_dictionaries.xml | 4 + .../test_dictionaries_dependency_xml/test.py | 6 +- .../configs/enable_dictionaries.xml | 4 + .../test_dictionaries_mysql/test.py | 5 +- .../configs/enable_dictionaries.xml | 4 + .../test_dictionaries_null_value/test.py | 6 +- .../configs/enable_dictionaries.xml | 4 + .../test_dictionaries_select_all/test.py | 6 +- .../configs/enable_dictionaries.xml | 4 + .../test.py | 5 +- .../configs/enable_dictionaries.xml | 4 + .../test_default_reading.py | 6 +- .../test_default_string.py | 6 +- .../test_dict_get.py | 5 +- .../test_dict_get_or_default.py | 5 +- .../configs/enable_dictionaries.xml | 4 + .../test_dictionary_custom_settings/test.py | 7 +- .../test_dictionary_ddl_on_cluster/test.py | 8 +- .../test_disk_types/configs/storage.xml | 16 ++ tests/integration/test_disk_types/test.py | 3 +- .../test_distributed_ddl/cluster.py | 15 +- .../configs_secure/config.d/ssl_conf.xml | 5 +- .../test_distributed_ddl_password/test.py | 12 +- .../test_distributed_format/test.py | 2 +- .../configs_secure/config.d/ssl_conf.xml | 5 +- .../test.py | 12 +- .../test.py | 2 +- .../test_enabling_access_management/test.py | 2 +- .../test_extreme_deduplication/test.py | 4 +- .../configs/dhparam.pem | 8 + .../configs/ssl_conf.xml | 6 +- .../test_https_replication/test.py | 12 +- .../test_log_family_s3/configs/minio.xml | 13 ++ .../test_log_family_s3/configs/ssl.xml | 12 ++ tests/integration/test_log_family_s3/test.py | 2 +- .../test.py | 10 +- tests/integration/test_merge_tree_s3/test.py | 4 +- .../configs/config.d/query_log.xml | 9 + .../configs/config.d/ssl_conf.xml | 12 ++ .../test_merge_tree_s3_with_cache/test.py | 4 +- tests/integration/test_multiple_disks/test.py | 6 +- .../test_mysql_database_engine/test.py | 3 +- .../test_mysql_protocol/configs/log_conf.xml | 10 ++ .../test_mysql_protocol/configs/mysql.xml | 4 + .../test_mysql_protocol/configs/ssl_conf.xml | 18 ++ tests/integration/test_mysql_protocol/test.py | 17 +- .../configs/enable_dictionaries.xml | 4 + .../configs/odbc_logging.xml | 8 + .../test_odbc_interaction/configs/openssl.xml | 12 ++ .../integration/test_odbc_interaction/test.py | 5 +- tests/integration/test_old_versions/test.py | 15 +- .../test_polymorphic_parts/test.py | 16 +- .../configs/default_passwd.xml | 13 ++ .../test_postgresql_protocol/configs/log.xml | 10 ++ .../configs/postresql.xml | 4 + .../configs/ssl_conf.xml | 18 ++ .../test_postgresql_protocol/test.py | 5 +- .../test_profile_events_s3/configs/log.xml | 10 ++ .../configs/query_log.xml | 9 + .../configs/ssl_conf.xml | 12 ++ .../test_profile_events_s3/test.py | 2 +- tests/integration/test_quorum_inserts/test.py | 9 +- tests/integration/test_quota/test.py | 170 +++++++++--------- tests/integration/test_random_inserts/test.py | 4 +- .../configs/max_table_size_to_drop.xml | 5 + .../test.py | 5 +- tests/integration/test_rename_column/test.py | 5 +- .../test_replicated_merge_tree_s3/test.py | 6 +- .../configs/users.d/another_user.xml | 13 ++ .../any_join_distinct_right_table_keys.xml | 8 + tests/integration/test_row_policy/test.py | 10 +- .../configs/config.d/ssl.xml | 12 ++ tests/integration/test_s3_with_https/test.py | 2 +- tests/integration/test_s3_with_proxy/test.py | 2 +- .../test_settings_constraints/test.py | 3 +- .../test.py | 8 +- tests/integration/test_storage_hdfs/test.py | 2 +- .../integration/test_storage_rabbitmq/test.py | 1 - .../dictionary_clickhouse_cache.xml | 4 +- .../dictionary_clickhouse_flat.xml | 4 +- tests/integration/test_system_queries/test.py | 7 +- tests/integration/test_text_log_level/test.py | 2 +- tests/integration/test_tmp_policy/test.py | 2 +- tests/integration/test_ttl_move/test.py | 8 +- .../test_user_ip_restrictions/test.py | 18 +- .../test_user_zero_database_access.py | 2 +- .../configs_secure/conf.d/ssl_conf.xml | 4 +- .../integration/test_zookeeper_config/test.py | 22 +-- 117 files changed, 922 insertions(+), 439 deletions(-) create mode 100644 tests/integration/helpers/0_common_enable_dictionaries.xml create mode 120000 tests/integration/helpers/dictonaries create mode 100644 tests/integration/test_dictionaries_all_layouts_and_sources/configs/disable_ssl_verification.xml create mode 100644 tests/integration/test_dictionaries_all_layouts_and_sources/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionaries_ddl/configs/allow_remote_node.xml create mode 100644 tests/integration/test_dictionaries_ddl/configs/dictionaries/conflict_name_dictionary.xml create mode 100644 tests/integration/test_dictionaries_ddl/configs/dictionaries/lazy_load_dictionary.xml create mode 100644 tests/integration/test_dictionaries_ddl/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionaries_ddl/configs/user_admin.xml create mode 100644 tests/integration/test_dictionaries_dependency_xml/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionaries_mysql/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionaries_null_value/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionaries_select_all/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionaries_update_and_reload/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionary_allow_read_expired_keys/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_dictionary_custom_settings/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_disk_types/configs/storage.xml create mode 100644 tests/integration/test_https_replication/configs/dhparam.pem create mode 100644 tests/integration/test_log_family_s3/configs/minio.xml create mode 100644 tests/integration/test_log_family_s3/configs/ssl.xml create mode 100644 tests/integration/test_merge_tree_s3_with_cache/configs/config.d/query_log.xml create mode 100644 tests/integration/test_merge_tree_s3_with_cache/configs/config.d/ssl_conf.xml create mode 100644 tests/integration/test_mysql_protocol/configs/log_conf.xml create mode 100644 tests/integration/test_mysql_protocol/configs/mysql.xml create mode 100644 tests/integration/test_mysql_protocol/configs/ssl_conf.xml create mode 100644 tests/integration/test_odbc_interaction/configs/enable_dictionaries.xml create mode 100644 tests/integration/test_odbc_interaction/configs/odbc_logging.xml create mode 100644 tests/integration/test_odbc_interaction/configs/openssl.xml create mode 100644 tests/integration/test_postgresql_protocol/configs/default_passwd.xml create mode 100644 tests/integration/test_postgresql_protocol/configs/log.xml create mode 100644 tests/integration/test_postgresql_protocol/configs/postresql.xml create mode 100644 tests/integration/test_postgresql_protocol/configs/ssl_conf.xml create mode 100644 tests/integration/test_profile_events_s3/configs/log.xml create mode 100644 tests/integration/test_profile_events_s3/configs/query_log.xml create mode 100644 tests/integration/test_profile_events_s3/configs/ssl_conf.xml create mode 100644 tests/integration/test_reload_max_table_size_to_drop/configs/max_table_size_to_drop.xml create mode 100644 tests/integration/test_row_policy/configs/users.d/another_user.xml create mode 100644 tests/integration/test_row_policy/configs/users.d/any_join_distinct_right_table_keys.xml create mode 100644 tests/integration/test_s3_with_https/configs/config.d/ssl.xml diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index c38260279ed..9abf3bde53d 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -19,7 +19,8 @@ set -e echo "Start tests" export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/clickhouse export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse -export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-config +export CLICKHOUSE_TESTS_CONFIG_DIR=/clickhouse-config +export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-base-config export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} diff --git a/tests/integration/CMakeLists.txt b/tests/integration/CMakeLists.txt index 8280464051f..f57ade79471 100644 --- a/tests/integration/CMakeLists.txt +++ b/tests/integration/CMakeLists.txt @@ -18,7 +18,7 @@ if(MAKE_STATIC_LIBRARIES AND DOCKER_CMD) if(NOT INTEGRATION_USE_RUNNER AND DOCKER_COMPOSE_CMD AND PYTEST_CMD) # To run one test with debug: # cmake . -DPYTEST_OPT="-ss;test_cluster_copier" - add_test(NAME integration-pytest WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} COMMAND env ${TEST_USE_BINARIES} "CLICKHOUSE_TESTS_BASE_CONFIG_DIR=${ClickHouse_SOURCE_DIR}/programs/server/" ${PYTEST_STARTER} ${PYTEST_CMD} ${PYTEST_OPT}) + add_test(NAME integration-pytest WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} COMMAND env ${TEST_USE_BINARIES} "CLICKHOUSE_TESTS_BASE_CONFIG_DIR=${ClickHouse_SOURCE_DIR}/programs/server/" "CLICKHOUSE_TESTS_CONFIG_DIR=${ClickHouse_SOURCE_DIR}/tests/config/" ${PYTEST_STARTER} ${PYTEST_CMD} ${PYTEST_OPT}) message(STATUS "Using tests in docker DOCKER=${DOCKER_CMD}; DOCKER_COMPOSE=${DOCKER_COMPOSE_CMD}; PYTEST=${PYTEST_STARTER} ${PYTEST_CMD} ${PYTEST_OPT}") endif() endif() diff --git a/tests/integration/helpers/0_common_enable_dictionaries.xml b/tests/integration/helpers/0_common_enable_dictionaries.xml new file mode 100644 index 00000000000..b6e52983db2 --- /dev/null +++ b/tests/integration/helpers/0_common_enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/dictionaries/*.xml + diff --git a/tests/integration/helpers/client.py b/tests/integration/helpers/client.py index 0ca6a977868..d88a21fbe46 100644 --- a/tests/integration/helpers/client.py +++ b/tests/integration/helpers/client.py @@ -71,7 +71,7 @@ class CommandRequest: self.stderr_file = tempfile.TemporaryFile() self.ignore_error = ignore_error - #print " ".join(command) + print " ".join(command) # we suppress stderror on client becase sometimes thread sanitizer # can print some debug information there diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index f421f979947..69db0c0fb10 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1,25 +1,25 @@ import base64 +import cassandra.cluster import distutils.dir_util +import docker import errno +import httplib +import logging import os import os.path as p +import pprint +import psycopg2 import pwd +import pymongo +import pymysql import re +import requests import shutil import socket import subprocess import time import urllib -import httplib -import requests import xml.dom.minidom -import logging -import docker -import pprint -import psycopg2 -import pymongo -import pymysql -import cassandra.cluster from dicttoxml import dicttoxml from kazoo.client import KazooClient from kazoo.exceptions import KazooException @@ -40,6 +40,7 @@ SANITIZER_SIGN = "==================" def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): full_path = os.path.join(path, fname) with open(full_path, 'w') as f: + f.write('TSAN_OPTIONS="external_symbolizer_path=/usr/bin/llvm-symbolizer"\n') for var, value in variables.items(): f.write("=".join([var, value]) + "\n") return full_path @@ -88,12 +89,16 @@ class ClickHouseCluster: these directories will contain logs, database files, docker-compose config, ClickHouse configs etc. """ - def __init__(self, base_path, name=None, base_configs_dir=None, server_bin_path=None, client_bin_path=None, + def __init__(self, base_path, name=None, base_config_dir=None, config_dir=None, server_bin_path=None, client_bin_path=None, odbc_bridge_bin_path=None, zookeeper_config_path=None, custom_dockerd_host=None): + for param in os.environ.keys(): + print "ENV %40s %s" % (param,os.environ[param]) self.base_dir = p.dirname(base_path) self.name = name if name is not None else '' - self.base_configs_dir = base_configs_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR', + self.base_config_dir = base_config_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR', + '/etc/clickhouse-server/') + self.config_dir = config_dir or os.environ.get('CLICKHOUSE_TESTS_CONFIG_DIR', '/etc/clickhouse-server/') self.server_bin_path = p.realpath( server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse')) @@ -154,6 +159,7 @@ class ClickHouseCluster: self.docker_client = None self.is_up = False + print "CLUSTER INIT base_config_dir:{} config_dir:{}".format(self.base_config_dir, self.config_dir) def get_client_cmd(self): cmd = self.client_bin_path @@ -161,7 +167,7 @@ class ClickHouseCluster: cmd += " client" return cmd - def add_instance(self, name, config_dir=None, main_configs=None, user_configs=None, macros=None, + def add_instance(self, name, base_config_dir=None, config_dir=None, main_configs=None, user_configs=None, dictionaries = None, macros=None, 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, @@ -172,6 +178,7 @@ class ClickHouseCluster: name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. config_dir - a directory with config files which content will be copied to /etc/clickhouse-server/ directory + base_config_dir - a directory with config.xml and users.xml files which will be copied to /etc/clickhouse-server/ directory main_configs - a list of config files that will be added to config.d/ directory user_configs - a list of config files that will be added to users.d/ directory with_zookeeper - if True, add ZooKeeper configuration to configs and ZooKeeper instances to the cluster. @@ -184,11 +191,11 @@ class ClickHouseCluster: raise Exception("Can\'t add instance `%s': there is already an instance with the same name!" % name) instance = ClickHouseInstance( - self, self.base_dir, name, config_dir, main_configs or [], user_configs or [], macros or {}, - with_zookeeper, + self, self.base_dir, name, base_config_dir if base_config_dir else self.base_config_dir, + config_dir if config_dir else self.config_dir, main_configs or [], user_configs or [], dictionaries or [], + macros or {}, with_zookeeper, self.zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra, - self.base_configs_dir, self.server_bin_path, - self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname, + self.server_bin_path, self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname, env_variables=env_variables or {}, image=image, stay_alive=stay_alive, ipv4_address=ipv4_address, ipv6_address=ipv6_address, with_installed_binary=with_installed_binary, tmpfs=tmpfs or []) @@ -458,19 +465,19 @@ class ClickHouseCluster: try: minio_client.list_buckets() - logging.info("Connected to Minio.") + print("Connected to Minio.") if minio_client.bucket_exists(self.minio_bucket): minio_client.remove_bucket(self.minio_bucket) minio_client.make_bucket(self.minio_bucket) - logging.info("S3 bucket '%s' created", self.minio_bucket) + print("S3 bucket '%s' created", self.minio_bucket) self.minio_client = minio_client return except Exception as ex: - logging.warning("Can't connect to Minio: %s", str(ex)) + print("Can't connect to Minio: %s", str(ex)) time.sleep(1) raise Exception("Can't wait Minio to start") @@ -482,10 +489,10 @@ class ClickHouseCluster: try: sr_client._send_request(sr_client.url) self.schema_registry_client = sr_client - logging.info("Connected to SchemaRegistry") + print("Connected to SchemaRegistry") return except Exception as ex: - logging.warning("Can't connect to SchemaRegistry: %s", str(ex)) + print("Can't connect to SchemaRegistry: %s", str(ex)) time.sleep(1) def wait_cassandra_to_start(self, timeout=30): @@ -501,25 +508,27 @@ class ClickHouseCluster: time.sleep(1) def start(self, destroy_dirs=True): + print "Cluster start called. is_up={}, destroy_dirs={}".format(self.is_up, destroy_dirs) if self.is_up: return # Just in case kill unstopped containers from previous launch try: - logging.info("Trying to kill unstopped containers...") + print("Trying to kill unstopped containers...") if not subprocess_call(['docker-compose', 'kill']): subprocess_call(['docker-compose', 'down', '--volumes']) - logging.info("Unstopped containers killed") + print("Unstopped containers killed") except: pass try: if destroy_dirs and p.exists(self.instances_dir): - logging.info("Removing instances dir %s", self.instances_dir) + print("Removing instances dir %s", self.instances_dir) shutil.rmtree(self.instances_dir) for instance in self.instances.values(): + print('Setup directory for instance: {} destroy_dirs: {}'.format(instance.name, destroy_dirs)) instance.create_dir(destroy_dir=destroy_dirs) self.docker_client = docker.from_env(version=self.docker_api_version) @@ -527,6 +536,7 @@ class ClickHouseCluster: common_opts = ['up', '-d', '--force-recreate'] if self.with_zookeeper and self.base_zookeeper_cmd: + print('Setup ZooKeeper') env = os.environ.copy() if not self.zookeeper_use_tmpfs: env['ZK_FS'] = 'bind' @@ -545,14 +555,17 @@ class ClickHouseCluster: self.wait_zookeeper_to_start(120) if self.with_mysql and self.base_mysql_cmd: + print('Setup MySQL') subprocess_check_call(self.base_mysql_cmd + common_opts) self.wait_mysql_to_start(120) if self.with_postgres and self.base_postgres_cmd: + print('Setup Postgres') subprocess_check_call(self.base_postgres_cmd + common_opts) self.wait_postgres_to_start(120) if self.with_kafka and self.base_kafka_cmd: + print('Setup Kafka') subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes']) self.kafka_docker_id = self.get_instance_docker_id('kafka1') self.wait_schema_registry_to_start(120) @@ -562,14 +575,17 @@ class ClickHouseCluster: self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1') if self.with_hdfs and self.base_hdfs_cmd: + print('Setup HDFS') subprocess_check_call(self.base_hdfs_cmd + common_opts) self.wait_hdfs_to_start(120) if self.with_mongo and self.base_mongo_cmd: + print('Setup Mongo') subprocess_check_call(self.base_mongo_cmd + common_opts) self.wait_mongo_to_start(30) if self.with_redis and self.base_redis_cmd: + print('Setup Redis') subprocess_check_call(self.base_redis_cmd + ['up', '-d', '--force-recreate']) time.sleep(10) @@ -608,18 +624,19 @@ class ClickHouseCluster: self.wait_cassandra_to_start() clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate'] - logging.info("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd))) + print("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd))) subprocess_check_call(clickhouse_start_cmd) - logging.info("ClickHouse instance created") + print("ClickHouse instance created") + start_deadline = time.time() + 20.0 # seconds for instance in self.instances.itervalues(): instance.docker_client = self.docker_client instance.ip_address = self.get_instance_ip(instance.name) - logging.info("Waiting for ClickHouse start...") + print("Waiting for ClickHouse start...") instance.wait_for_start(start_deadline) - logging.info("ClickHouse started") + print("ClickHouse started") instance.client = Client(instance.ip_address, command=self.client_bin_path) @@ -633,7 +650,10 @@ class ClickHouseCluster: def shutdown(self, kill=True): sanitizer_assert_instance = None with open(self.docker_logs_path, "w+") as f: - subprocess.check_call(self.base_cmd + ['logs'], stdout=f) + try: + subprocess.check_call(self.base_cmd + ['logs'], stdout=f) + except Exception as e: + print "Unable to get logs from docker." f.seek(0) for line in f: if SANITIZER_SIGN in line: @@ -641,8 +661,15 @@ class ClickHouseCluster: break if kill: - subprocess_check_call(self.base_cmd + ['kill']) - subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans']) + try: + subprocess_check_call(self.base_cmd + ['kill']) + except Exception as e: + print "Kill command failed durung shutdown. {}".format(repr(e)) + + try: + subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans']) + except Exception as e: + print "Down + remove orphans failed durung shutdown. {}".format(repr(e)) self.is_up = False @@ -707,7 +734,7 @@ services: image: {image} hostname: {hostname} volumes: - - {configs_dir}:/etc/clickhouse-server/ + - {instance_config_dir}:/etc/clickhouse-server/ - {db_dir}:/var/lib/clickhouse/ - {logs_dir}:/var/log/clickhouse-server/ {binary_volume} @@ -723,6 +750,9 @@ services: - {env_file} security_opt: - label:disable + dns_opt: + - timeout:1 + - attempts:3 {networks} {app_net} {ipv4_address} @@ -735,9 +765,9 @@ services: class ClickHouseInstance: def __init__( - self, cluster, base_path, name, custom_config_dir, custom_main_configs, custom_user_configs, macros, - with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra, - base_configs_dir, server_bin_path, odbc_bridge_bin_path, + self, cluster, base_path, name, base_config_dir, config_dir, custom_main_configs, custom_user_configs, custom_dictionaries, + 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, image="yandex/clickhouse-integration-test", stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None): @@ -749,15 +779,16 @@ class ClickHouseInstance: self.hostname = hostname if hostname is not None else self.name self.tmpfs = tmpfs or [] - self.custom_config_dir = p.abspath(p.join(base_path, custom_config_dir)) if custom_config_dir else None + self.base_config_dir = p.abspath(p.join(base_path, base_config_dir)) if base_config_dir else None + self.config_dir = p.abspath(p.join(base_path, config_dir)) if config_dir else None self.custom_main_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_main_configs] 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.macros = macros if macros is not None else {} self.with_zookeeper = with_zookeeper self.zookeeper_config_path = zookeeper_config_path - self.base_configs_dir = base_configs_dir self.server_bin_path = server_bin_path self.odbc_bridge_bin_path = odbc_bridge_bin_path @@ -773,7 +804,7 @@ class ClickHouseInstance: self.docker_compose_path = p.join(self.path, 'docker_compose.yml') self.env_variables = env_variables or {} if with_odbc_drivers: - self.odbc_ini_path = os.path.dirname(self.docker_compose_path) + "/odbc.ini:/etc/odbc.ini" + self.odbc_ini_path = self.path + "/odbc.ini:/etc/odbc.ini" self.with_mysql = True else: self.odbc_ini_path = "" @@ -975,7 +1006,7 @@ class ClickHouseInstance: time_left = deadline - current_time if deadline is not None and current_time >= deadline: raise Exception("Timed out while waiting for instance `{}' with ip address {} to start. " - "Container status: {}".format(self.name, self.ip_address, status)) + "Container status: {}, logs: {}".format(self.name, self.ip_address, status, handle.logs())) # Repeatedly poll the instance address until there is something that listens there. # Usually it means that ClickHouse is ready to accept queries. @@ -1057,40 +1088,50 @@ class ClickHouseInstance: os.makedirs(self.path) - configs_dir = p.abspath(p.join(self.path, 'configs')) - os.mkdir(configs_dir) + instance_config_dir = p.abspath(p.join(self.path, 'configs')) + os.makedirs(instance_config_dir) - shutil.copy(p.join(self.base_configs_dir, 'config.xml'), configs_dir) - shutil.copy(p.join(self.base_configs_dir, 'users.xml'), configs_dir) + print "Copy common default production configuration from {}".format(self.base_config_dir) + shutil.copyfile(p.join(self.base_config_dir, 'config.xml'), p.join(instance_config_dir, 'config.xml')) + shutil.copyfile(p.join(self.base_config_dir, 'users.xml'), p.join(instance_config_dir, 'users.xml')) + print "Create directory for configuration generated in this helper" # used by all utils with any config - conf_d_dir = p.abspath(p.join(configs_dir, 'conf.d')) - # used by server with main config.xml - self.config_d_dir = p.abspath(p.join(configs_dir, 'config.d')) - users_d_dir = p.abspath(p.join(configs_dir, 'users.d')) + conf_d_dir = p.abspath(p.join(instance_config_dir, 'conf.d')) os.mkdir(conf_d_dir) - os.mkdir(self.config_d_dir) - os.mkdir(users_d_dir) + print "Create directory for common tests configuration" + # used by server with main config.xml + self.config_d_dir = p.abspath(p.join(instance_config_dir, 'config.d')) + os.mkdir(self.config_d_dir) + users_d_dir = p.abspath(p.join(instance_config_dir, 'users.d')) + os.mkdir(users_d_dir) + dictionaries_dir = p.abspath(p.join(instance_config_dir, 'dictionaries')) + os.mkdir(dictionaries_dir) + + print "Copy common configuration from helpers" # The file is named with 0_ prefix to be processed before other configuration overloads. shutil.copy(p.join(HELPERS_DIR, '0_common_instance_config.xml'), self.config_d_dir) shutil.copy(p.join(HELPERS_DIR, '0_common_instance_users.xml'), users_d_dir) + if len(self.custom_dictionaries_paths): + shutil.copy(p.join(HELPERS_DIR, '0_common_enable_dictionaries.xml'), self.config_d_dir) - # Generate and write macros file + print "Generate and write macros file" macros = self.macros.copy() macros['instance'] = self.name - with open(p.join(self.config_d_dir, 'macros.xml'), 'w') as macros_config: + with open(p.join(conf_d_dir, 'macros.xml'), 'w') as macros_config: macros_config.write(self.dict_to_xml({"macros": macros})) # Put ZooKeeper config if self.with_zookeeper: shutil.copy(self.zookeeper_config_path, conf_d_dir) - # Copy config dir - if self.custom_config_dir: - distutils.dir_util.copy_tree(self.custom_config_dir, configs_dir) + # print "Copy config dir {} to {}".format(self.config_dir, instance_config_dir) + # if self.config_dir: + # distutils.dir_util.copy_tree(self.config_dir, instance_config_dir) # 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: shutil.copy(path, self.config_d_dir) @@ -1098,12 +1139,21 @@ class ClickHouseInstance: for path in self.custom_user_config_paths: shutil.copy(path, users_d_dir) + + self.config_dir + # Copy dictionaries configs to configs/dictionaries + for path in self.custom_dictionaries_paths: + shutil.copy(path, dictionaries_dir) + db_dir = p.abspath(p.join(self.path, 'database')) + print "Setup database dir {}".format(db_dir) os.mkdir(db_dir) if self.clickhouse_path_dir is not None: + print "Database files taken from {}".format(self.clickhouse_path_dir) distutils.dir_util.copy_tree(self.clickhouse_path_dir, db_dir) logs_dir = p.abspath(p.join(self.path, 'logs')) + print "Setup logs dir {}".format(logs_dir) os.mkdir(logs_dir) depends_on = [] @@ -1128,6 +1178,8 @@ class ClickHouseInstance: env_file = _create_env_file(os.path.dirname(self.docker_compose_path), self.env_variables) + print "Env {} stored in {}".format(self.env_variables, env_file) + odbc_ini_path = "" if self.odbc_ini_path: self._create_odbc_config_file() @@ -1138,6 +1190,8 @@ class ClickHouseInstance: if self.stay_alive: entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND + print "Entrypoint cmd: {}".format(entrypoint_cmd) + networks = app_net = ipv4_address = ipv6_address = net_aliases = net_alias1 = "" if self.ipv4_address is not None or self.ipv6_address is not None or self.hostname != self.name: networks = "networks:" @@ -1157,6 +1211,7 @@ class ClickHouseInstance: binary_volume = "- " + self.server_bin_path + ":/usr/share/clickhouse_fresh" odbc_bridge_volume = "- " + self.odbc_bridge_bin_path + ":/usr/share/clickhouse-odbc-bridge_fresh" + with open(self.docker_compose_path, 'w') as docker_compose: docker_compose.write(DOCKER_COMPOSE_TEMPLATE.format( image=self.image, @@ -1164,7 +1219,7 @@ class ClickHouseInstance: hostname=self.hostname, binary_volume=binary_volume, odbc_bridge_volume=odbc_bridge_volume, - configs_dir=configs_dir, + instance_config_dir=instance_config_dir, config_d_dir=self.config_d_dir, db_dir=db_dir, tmpfs=str(self.tmpfs), diff --git a/tests/integration/helpers/dictonaries b/tests/integration/helpers/dictonaries new file mode 120000 index 00000000000..b33ab3b1e87 --- /dev/null +++ b/tests/integration/helpers/dictonaries @@ -0,0 +1 @@ +../../config/dict_examples/ \ No newline at end of file diff --git a/tests/integration/helpers/test_tools.py b/tests/integration/helpers/test_tools.py index 93265d280df..67ca025c58a 100644 --- a/tests/integration/helpers/test_tools.py +++ b/tests/integration/helpers/test_tools.py @@ -11,6 +11,9 @@ class TSV: raw_lines = contents.splitlines(True) elif isinstance(contents, list): raw_lines = ['\t'.join(map(str, l)) if isinstance(l, list) else str(l) for l in contents] + elif isinstance(contents, TSV): + self.lines = contents.lines + return else: raise TypeError("contents must be either file or string or list, actual type: " + type(contents).__name__) self.lines = [l.strip() for l in raw_lines if l.strip()] diff --git a/tests/integration/runner b/tests/integration/runner index e5d6eabe794..0b084d88f9a 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -53,7 +53,7 @@ def check_args_and_update_paths(args): logging.info("base_configs_dir: {}, binary: {}, cases_dir: {} ".format(args.base_configs_dir, args.binary, args.cases_dir)) - for path in [args.binary, args.base_configs_dir, args.cases_dir, CLICKHOUSE_ROOT]: + for path in [args.binary, args.bridge_binary, args.base_configs_dir, args.cases_dir, CLICKHOUSE_ROOT]: if not os.path.exists(path): raise Exception("Path {} doesn't exist".format(path)) diff --git a/tests/integration/test_access_control_on_cluster/test.py b/tests/integration/test_access_control_on_cluster/test.py index 07c72e94be0..9f053afb607 100644 --- a/tests/integration/test_access_control_on_cluster/test.py +++ b/tests/integration/test_access_control_on_cluster/test.py @@ -4,9 +4,9 @@ from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) -ch1 = cluster.add_instance('ch1', config_dir="configs", with_zookeeper=True) -ch2 = cluster.add_instance('ch2', config_dir="configs", with_zookeeper=True) -ch3 = cluster.add_instance('ch3', config_dir="configs", with_zookeeper=True) +ch1 = cluster.add_instance('ch1', main_configs=["configs/config.d/clusters.xml"], with_zookeeper=True) +ch2 = cluster.add_instance('ch2', main_configs=["configs/config.d/clusters.xml"], with_zookeeper=True) +ch3 = cluster.add_instance('ch3', main_configs=["configs/config.d/clusters.xml"], with_zookeeper=True) @pytest.fixture(scope="module", autouse=True) def started_cluster(): diff --git a/tests/integration/test_adaptive_granularity/test.py b/tests/integration/test_adaptive_granularity/test.py index 671cb5a672b..9feef62f799 100644 --- a/tests/integration/test_adaptive_granularity/test.py +++ b/tests/integration/test_adaptive_granularity/test.py @@ -9,23 +9,23 @@ from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) -node2 = cluster.add_instance('node2', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) -node3 = cluster.add_instance('node3', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', with_installed_binary=True) -node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node3 = cluster.add_instance('node3', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', with_installed_binary=True) +node4 = cluster.add_instance('node4', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) -node5 = cluster.add_instance('node5', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', with_installed_binary=True) -node6 = cluster.add_instance('node6', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node5 = cluster.add_instance('node5', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', with_installed_binary=True) +node6 = cluster.add_instance('node6', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) -node7 = cluster.add_instance('node7', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', stay_alive=True, with_installed_binary=True) -node8 = cluster.add_instance('node8', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) +node7 = cluster.add_instance('node7', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', stay_alive=True, with_installed_binary=True) +node8 = cluster.add_instance('node8', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) -node9 = cluster.add_instance('node9', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/merge_tree_settings.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) -node10 = cluster.add_instance('node10', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/merge_tree_settings.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', stay_alive=True, with_installed_binary=True) +node9 = cluster.add_instance('node9', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/merge_tree_settings.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) +node10 = cluster.add_instance('node10', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/merge_tree_settings.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', stay_alive=True, with_installed_binary=True) -node11 = cluster.add_instance('node11', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) -node12 = cluster.add_instance('node12', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) +node11 = cluster.add_instance('node11', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) +node12 = cluster.add_instance('node12', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) def prepare_single_pair_with_setting(first_node, second_node, group): diff --git a/tests/integration/test_allowed_client_hosts/test.py b/tests/integration/test_allowed_client_hosts/test.py index 23f7f0a4abd..f187b6d889c 100644 --- a/tests/integration/test_allowed_client_hosts/test.py +++ b/tests/integration/test_allowed_client_hosts/test.py @@ -4,7 +4,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -server = cluster.add_instance('server', config_dir="configs") +server = cluster.add_instance('server', user_configs=["configs/users.d/network.xml"]) clientA1 = cluster.add_instance('clientA1', hostname = 'clientA1.com') clientA2 = cluster.add_instance('clientA2', hostname = 'clientA2.com') @@ -20,7 +20,12 @@ clientD2 = cluster.add_instance('clientD2', hostname = 'xxx.clientD0002.ru') clientD3 = cluster.add_instance('clientD3', hostname = 'clientD0003.ru') +def check_clickhouse_is_ok(client_node, server_node): + assert client_node.exec_in_container(["bash", "-c", "/usr/bin/curl -s {}:8123 ".format(server_node.hostname)]) == "Ok.\n" + + def query_from_one_node_to_another(client_node, server_node, query): + check_clickhouse_is_ok(client_node, server_node) return client_node.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host {} --query {!r}".format(server_node.hostname, query)]) @@ -56,5 +61,6 @@ def test_allowed_host(): for client_node in expected_to_fail: with pytest.raises(Exception) as e: - query_from_one_node_to_another(client_node, server, "SELECT * FROM test_table") + result = query_from_one_node_to_another(client_node, server, "SELECT * FROM test_table") + print("Client node: {} Server node: {} Result: {}".format(client_node, server_node, result)) assert "default: Authentication failed" in str(e) diff --git a/tests/integration/test_allowed_url_from_config/test.py b/tests/integration/test_allowed_url_from_config/test.py index 688f94cb058..2a666e4e2ec 100644 --- a/tests/integration/test_allowed_url_from_config/test.py +++ b/tests/integration/test_allowed_url_from_config/test.py @@ -40,7 +40,7 @@ def test_config_with_only_regexp_hosts(start_cluster): assert node3.query("CREATE TABLE table_test_3_1 (word String) Engine=URL('https://host:80', HDFS)") == "" assert node3.query("CREATE TABLE table_test_3_2 (word String) Engine=URL('https://yandex.ru', CSV)") == "" assert "not allowed" in node3.query_and_get_error("CREATE TABLE table_test_3_3 (word String) Engine=URL('https://host', CSV)") - assert "not allowed" in node3.query_and_get_error("CREATE TABLE table_test_3_4 (word String) Engine=URL('https://yandex2.ru', S3)") + assert "not allowed" in node3.query_and_get_error("CREATE TABLE table_test_3_4 (word String) Engine=URL('https://yandex2.ru', S3)") def test_config_without_allowed_hosts(start_cluster): assert node4.query("CREATE TABLE table_test_4_1 (word String) Engine=URL('https://host:80', CSV)") == "" @@ -49,18 +49,18 @@ def test_config_without_allowed_hosts(start_cluster): assert node4.query("CREATE TABLE table_test_4_4 (word String) Engine=URL('ftp://something.com', S3)") == "" def test_table_function_remote(start_cluster): + assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-{1|2}', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1}) + assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-1,example01-02-1', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1}) + assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remote('example01-0{1,2}-1', system, events", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1}) + assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remote('example01-0{1,2}-{1|2}', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1}) + assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-{01..02}-{1|2}', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1}) + assert "not allowed" in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-1,example01-03-1', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1}) + assert "not allowed" in node6.query_and_get_error("SELECT * FROM remote('example01-01-{1|3}', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1}) + assert "not allowed" in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-0{1,3}-1', system, metrics)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1}) assert node6.query("SELECT * FROM remote('localhost', system, events)") != "" assert node6.query("SELECT * FROM remoteSecure('localhost', system, metrics)") != "" assert "URL \"localhost:800\" is not allowed in config.xml" in node6.query_and_get_error("SELECT * FROM remoteSecure('localhost:800', system, events)") assert "URL \"localhost:800\" is not allowed in config.xml" in node6.query_and_get_error("SELECT * FROM remote('localhost:800', system, metrics)") - assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-1,example01-02-1', system, events)") - assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remote('example01-0{1,2}-1', system, events") - assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-{1|2}', system, events)") - assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remote('example01-0{1,2}-{1|2}', system, events)") - assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-{01..02}-{1|2}', system, events)") - assert "not allowed" in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-1,example01-03-1', system, events)") - assert "not allowed" in node6.query_and_get_error("SELECT * FROM remote('example01-01-{1|3}', system, events)") - assert "not allowed" in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-0{1,3}-1', system, metrics)") def test_redirect(start_cluster): hdfs_api = HDFSApi("root") diff --git a/tests/integration/test_atomic_drop_table/test.py b/tests/integration/test_atomic_drop_table/test.py index 279d13ac4da..ee79a3ff080 100644 --- a/tests/integration/test_atomic_drop_table/test.py +++ b/tests/integration/test_atomic_drop_table/test.py @@ -6,7 +6,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True) +node1 = cluster.add_instance('node1', main_configs=["configs/config.d/zookeeper_session_timeout.xml", "configs/remote_servers.xml"], with_zookeeper=True) @pytest.fixture(scope="module") diff --git a/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml b/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml index 54a8822fa98..632ab84d6a2 100644 --- a/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml +++ b/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml @@ -1,80 +1,74 @@ + - - - - - true - - s0_0_0 - 9000 - - - s0_0_1 - 9000 - - - - true - - s0_1_0 - 9000 - - - - - - - true - - s1_0_0 - 9000 - - - s1_0_1 - 9000 - - - - true - - s1_1_0 - 9000 - - - - - - - true - - s0_0_0 - 9000 - - - s0_0_1 - 9000 - - - - - - - - s0_0_0 - 9000 - - - - - - - - - s1_0_0 - 9000 - - - - - + + + + true + + s0_0_0 + 9000 + + + s0_0_1 + 9000 + + + + true + + s0_1_0 + 9000 + + + + + + true + + s1_0_0 + 9000 + + + s1_0_1 + 9000 + + + + true + + s1_1_0 + 9000 + + + + + + true + + s0_0_0 + 9000 + + + s0_0_1 + 9000 + + + + + + + s0_0_0 + 9000 + + + + + + + s1_0_0 + 9000 + + + + diff --git a/tests/integration/test_cluster_copier/test.py b/tests/integration/test_cluster_copier/test.py index 983cac596dc..3f9ca8a053c 100644 --- a/tests/integration/test_cluster_copier/test.py +++ b/tests/integration/test_cluster_copier/test.py @@ -54,7 +54,8 @@ def started_cluster(): for replica_name in replicas: name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name) cluster.add_instance(name, - config_dir="configs", + main_configs=["configs/conf.d/query_log.xml", "configs/conf.d/ddl.xml", "configs/conf.d/clusters.xml"], + user_configs=["configs/users.xml"], macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name}, with_zookeeper=True) @@ -226,6 +227,7 @@ def execute_task(task, cmd_options): zk.ensure_path(zk_task_path) zk.create(zk_task_path + "/description", task.copier_task_config) + # Run cluster-copier processes on each node docker_api = docker.from_env().api copiers_exec_ids = [] @@ -241,9 +243,11 @@ def execute_task(task, cmd_options): for instance_name in copiers: instance = cluster.instances[instance_name] container = instance.get_docker_handle() + instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs/config-copier.xml"), "/etc/clickhouse-server/config-copier.xml") + print "Copied copier config to {}".format(instance.name) exec_id = docker_api.exec_create(container.id, cmd, stderr=True) - docker_api.exec_start(exec_id, detach=True) - + output = docker_api.exec_start(exec_id).decode('utf8') + print(output) copiers_exec_ids.append(exec_id) print "Copier for {} ({}) has started".format(instance.name, instance.ip_address) diff --git a/tests/integration/test_cluster_copier/trivial_test.py b/tests/integration/test_cluster_copier/trivial_test.py index 70c66653cb2..1697f8bbdfa 100644 --- a/tests/integration/test_cluster_copier/trivial_test.py +++ b/tests/integration/test_cluster_copier/trivial_test.py @@ -34,7 +34,7 @@ def started_cluster(): for replica_name in replicas: name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name) cluster.add_instance(name, - config_dir="configs", + main_configs=[], user_configs=[], macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name}, with_zookeeper=True) diff --git a/tests/integration/test_config_corresponding_root/test.py b/tests/integration/test_config_corresponding_root/test.py index fd5d3eae3ff..1c714654820 100644 --- a/tests/integration/test_config_corresponding_root/test.py +++ b/tests/integration/test_config_corresponding_root/test.py @@ -4,10 +4,9 @@ import pytest from helpers.cluster import ClickHouseCluster SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -config_dir = os.path.join(SCRIPT_DIR, './configs') cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir = config_dir) +node = cluster.add_instance('node', main_configs=["configs/config.d/bad.xml"]) caught_exception = "" @pytest.fixture(scope="module") @@ -19,4 +18,5 @@ def start_cluster(): caught_exception = str(e) def test_work(start_cluster): + print(caught_exception) assert caught_exception.find("Root element doesn't have the corresponding root element as the config file.") != -1 diff --git a/tests/integration/test_custom_settings/test.py b/tests/integration/test_custom_settings/test.py index 444a4d21881..62c765a6ba0 100644 --- a/tests/integration/test_custom_settings/test.py +++ b/tests/integration/test_custom_settings/test.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir='configs') +node = cluster.add_instance('node', main_configs=["configs/config.d/text_log.xml"], user_configs=["configs/users.d/custom_settings.xml"]) @pytest.fixture(scope="module", autouse=True) diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/configs/disable_ssl_verification.xml b/tests/integration/test_dictionaries_all_layouts_and_sources/configs/disable_ssl_verification.xml new file mode 100644 index 00000000000..dc9958934d2 --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_and_sources/configs/disable_ssl_verification.xml @@ -0,0 +1,12 @@ + + + + + true + none + + AcceptCertificateHandler + + + + \ No newline at end of file diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_all_layouts_and_sources/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..8a3d6704670 --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_and_sources/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/*.xml + diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/test.py b/tests/integration/test_dictionaries_all_layouts_and_sources/test.py index f4b0ba9c1e4..4c35f9725a8 100644 --- a/tests/integration/test_dictionaries_all_layouts_and_sources/test.py +++ b/tests/integration/test_dictionaries_all_layouts_and_sources/test.py @@ -181,12 +181,18 @@ def setup_module(module): if not (field.is_key or field.is_range or field.is_range_key): DICTIONARIES_KV.append(get_dict(source, layout, field_keys + [field], field.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): - main_configs.append(os.path.join(dict_configs_path, fname)) - cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) - node = cluster.add_instance('node', main_configs=main_configs, with_mysql=True, with_mongo=True, with_redis=True, with_cassandra=True) - cluster.add_instance('clickhouse1') + 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") @@ -238,8 +244,8 @@ def remove_mysql_dicts(): 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")] + #global DICTIONARIES + #DICTIONARIES = [d for d in DICTIONARIES if not d.name.startswith("MySQL")] @pytest.mark.parametrize("fold", list(range(10))) diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..46d148ad9b9 --- /dev/null +++ b/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml @@ -0,0 +1,5 @@ + + + /etc/clickhouse-server/config.d/complex_key_cache_string.xml + /etc/clickhouse-server/config.d/ssd_complex_key_cache_string.xml + diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/test.py b/tests/integration/test_dictionaries_complex_key_cache_string/test.py index 2a62d66a5f8..8c676841f16 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/test.py +++ b/tests/integration/test_dictionaries_complex_key_cache_string/test.py @@ -5,13 +5,12 @@ from helpers.cluster import ClickHouseCluster @pytest.fixture(scope="function") def cluster(request): SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) - + cluster = ClickHouseCluster(__file__) try: if request.param == "memory": - node = cluster.add_instance('node', main_configs=['configs/dictionaries/complex_key_cache_string.xml']) + node = cluster.add_instance('node', main_configs=['configs/enable_dictionaries.xml', 'configs/dictionaries/complex_key_cache_string.xml']) if request.param == "ssd": - node = cluster.add_instance('node', main_configs=['configs/dictionaries/ssd_complex_key_cache_string.xml']) + node = cluster.add_instance('node', main_configs=['configs/enable_dictionaries.xml', 'configs/dictionaries/ssd_complex_key_cache_string.xml']) cluster.start() node.query("create table radars_table (radar_id String, radar_ip String, client_id String) engine=MergeTree() order by radar_id") diff --git a/tests/integration/test_dictionaries_ddl/configs/allow_remote_node.xml b/tests/integration/test_dictionaries_ddl/configs/allow_remote_node.xml new file mode 100644 index 00000000000..5e616865fef --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/configs/allow_remote_node.xml @@ -0,0 +1,5 @@ + + + node1 + + diff --git a/tests/integration/test_dictionaries_ddl/configs/dictionaries/conflict_name_dictionary.xml b/tests/integration/test_dictionaries_ddl/configs/dictionaries/conflict_name_dictionary.xml new file mode 100644 index 00000000000..75e6f8953eb --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/configs/dictionaries/conflict_name_dictionary.xml @@ -0,0 +1,41 @@ + + + test.conflicting_dictionary + + + localhost + 9000 + default + + test + xml_dictionary_table
+
+ + + + 0 + 0 + + + + 128 + + + + + id + + + SomeValue1 + UInt8 + 1 + + + + SomeValue2 + String + '' + + +
+
diff --git a/tests/integration/test_dictionaries_ddl/configs/dictionaries/lazy_load_dictionary.xml b/tests/integration/test_dictionaries_ddl/configs/dictionaries/lazy_load_dictionary.xml new file mode 100644 index 00000000000..d01f7a0155b --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/configs/dictionaries/lazy_load_dictionary.xml @@ -0,0 +1,4 @@ + + false + + diff --git a/tests/integration/test_dictionaries_ddl/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_ddl/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..ddb049db2a4 --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/configs/enable_dictionaries.xml @@ -0,0 +1,3 @@ + + /etc/clickhouse-server/config.d/*dictionary.xml + diff --git a/tests/integration/test_dictionaries_ddl/configs/user_admin.xml b/tests/integration/test_dictionaries_ddl/configs/user_admin.xml new file mode 100644 index 00000000000..3e53e05aee1 --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/configs/user_admin.xml @@ -0,0 +1,36 @@ + + + + + + + + + + + + ::/0 + + default + default + + default + test + + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/tests/integration/test_dictionaries_ddl/test.py b/tests/integration/test_dictionaries_ddl/test.py index 220aeb6998a..ff252401928 100644 --- a/tests/integration/test_dictionaries_ddl/test.py +++ b/tests/integration/test_dictionaries_ddl/test.py @@ -7,10 +7,10 @@ import warnings SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) -node1 = cluster.add_instance('node1', with_mysql=True, main_configs=['configs/dictionaries/simple_dictionary.xml']) -node2 = cluster.add_instance('node2', with_mysql=True, main_configs=['configs/dictionaries/simple_dictionary.xml', 'configs/dictionaries/lazy_load.xml']) -node3 = cluster.add_instance('node3', main_configs=['configs/dictionaries/dictionary_with_conflict_name.xml']) +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', with_mysql=True, main_configs=['configs/enable_dictionaries.xml','configs/dictionaries/simple_dictionary.xml'], user_configs=['configs/user_admin.xml']) +node2 = cluster.add_instance('node2', with_mysql=True, main_configs=['configs/allow_remote_node.xml','configs/enable_dictionaries.xml','configs/dictionaries/simple_dictionary.xml', 'configs/dictionaries/lazy_load_dictionary.xml'], user_configs=['configs/user_admin.xml']) +node3 = cluster.add_instance('node3', main_configs=['configs/allow_remote_node.xml','configs/enable_dictionaries.xml','configs/dictionaries/conflict_name_dictionary.xml'], user_configs=['configs/user_admin.xml']) def create_mysql_conn(user, password, hostname, port): @@ -49,7 +49,7 @@ def started_cluster(): (node2, 'complex_node2_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())'), (node2, 'complex_node2_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'), ]) -def test_crete_and_select_mysql(started_cluster, clickhouse, name, layout): +def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", 3308) execute_mysql_query(mysql_conn, "CREATE DATABASE IF NOT EXISTS clickhouse") execute_mysql_query(mysql_conn, "CREATE TABLE clickhouse.{} (key_field1 int, key_field2 bigint, value1 text, value2 float, PRIMARY KEY (key_field1, key_field2))".format(name)) @@ -93,8 +93,8 @@ def test_crete_and_select_mysql(started_cluster, clickhouse, name, layout): for i in range(172, 200): assert clickhouse.query("SELECT dictGetString('default.{}', 'value1', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)) == str(i) * 3 + '\n' - stroka = clickhouse.query("SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)).strip() - value = float(stroka) + string = clickhouse.query("SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)).strip() + value = float(string) assert int(value) == int(i * 2.718) clickhouse.query("select dictGetUInt8('xml_dictionary', 'SomeValue1', toUInt64(17))") == "17\n" diff --git a/tests/integration/test_dictionaries_dependency_xml/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_dependency_xml/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..89a4c99ef7a --- /dev/null +++ b/tests/integration/test_dictionaries_dependency_xml/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/dep_*.xml + diff --git a/tests/integration/test_dictionaries_dependency_xml/test.py b/tests/integration/test_dictionaries_dependency_xml/test.py index c0ce0af0313..da1146cd54c 100644 --- a/tests/integration/test_dictionaries_dependency_xml/test.py +++ b/tests/integration/test_dictionaries_dependency_xml/test.py @@ -3,11 +3,11 @@ import os from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml'] DICTIONARY_FILES = ['configs/dictionaries/dep_x.xml', 'configs/dictionaries/dep_y.xml', 'configs/dictionaries/dep_z.xml'] -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) -instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES) +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES,) @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionaries_mysql/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_mysql/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..76ed6af89ba --- /dev/null +++ b/tests/integration/test_dictionaries_mysql/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/mysql_dict*.xml + diff --git a/tests/integration/test_dictionaries_mysql/test.py b/tests/integration/test_dictionaries_mysql/test.py index 647e36c71b3..4d2a063e91d 100644 --- a/tests/integration/test_dictionaries_mysql/test.py +++ b/tests/integration/test_dictionaries_mysql/test.py @@ -8,10 +8,9 @@ import pymysql.cursors from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) CONFIG_FILES = ['configs/dictionaries/mysql_dict1.xml', 'configs/dictionaries/mysql_dict2.xml', 'configs/remote_servers.xml'] - -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) +CONFIG_FILES += ['configs/enable_dictionaries.xml'] +cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', main_configs=CONFIG_FILES, with_mysql = True) create_table_mysql_template = """ diff --git a/tests/integration/test_dictionaries_null_value/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_null_value/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..8a3d6704670 --- /dev/null +++ b/tests/integration/test_dictionaries_null_value/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/*.xml + diff --git a/tests/integration/test_dictionaries_null_value/test.py b/tests/integration/test_dictionaries_null_value/test.py index bb840d8f8f7..c4ad3782498 100644 --- a/tests/integration/test_dictionaries_null_value/test.py +++ b/tests/integration/test_dictionaries_null_value/test.py @@ -3,11 +3,11 @@ import os from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV, assert_eq_with_retry -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml'] DICTIONARY_FILES = ['configs/dictionaries/cache.xml'] -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) -instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES) +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES) @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionaries_select_all/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_select_all/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..fa26ed7ec3d --- /dev/null +++ b/tests/integration/test_dictionaries_select_all/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/dictionary*.xml + diff --git a/tests/integration/test_dictionaries_select_all/test.py b/tests/integration/test_dictionaries_select_all/test.py index 8bad8a9b214..7dc93b2df44 100644 --- a/tests/integration/test_dictionaries_select_all/test.py +++ b/tests/integration/test_dictionaries_select_all/test.py @@ -19,12 +19,12 @@ def setup_module(module): structure = generate_structure() dictionary_files = generate_dictionaries(os.path.join(SCRIPT_DIR, 'configs/dictionaries'), structure) - cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) - instance = cluster.add_instance('instance', main_configs=dictionary_files) + cluster = ClickHouseCluster(__file__) + instance = cluster.add_instance('instance', main_configs=dictionary_files+['configs/enable_dictionaries.xml']) test_table = DictionaryTestTable(os.path.join(SCRIPT_DIR, 'configs/dictionaries/source.tsv')) -@pytest.fixture(scope="module") +@pytest.fixture(scope="module", autouse=True) def started_cluster(): try: cluster.start() diff --git a/tests/integration/test_dictionaries_update_and_reload/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_update_and_reload/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..8a3d6704670 --- /dev/null +++ b/tests/integration/test_dictionaries_update_and_reload/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/*.xml + diff --git a/tests/integration/test_dictionaries_update_and_reload/test.py b/tests/integration/test_dictionaries_update_and_reload/test.py index 5e5c6514dd2..762fd3adc28 100644 --- a/tests/integration/test_dictionaries_update_and_reload/test.py +++ b/tests/integration/test_dictionaries_update_and_reload/test.py @@ -6,10 +6,11 @@ from helpers.client import QueryTimeoutExceedException from helpers.test_tools import assert_eq_with_retry SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml'] DICTIONARY_FILES = ['configs/dictionaries/cache_xypairs.xml', 'configs/dictionaries/executable.xml', 'configs/dictionaries/file.xml', 'configs/dictionaries/file.txt', 'configs/dictionaries/slow.xml'] -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) -instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES) +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES) @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/configs/enable_dictionaries.xml b/tests/integration/test_dictionary_allow_read_expired_keys/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..8a3d6704670 --- /dev/null +++ b/tests/integration/test_dictionary_allow_read_expired_keys/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/*.xml + diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py index 8da882679bd..b6b742c1de8 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py @@ -8,11 +8,11 @@ from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseKiller from helpers.network import PartitionManager -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) +cluster = ClickHouseCluster(__file__) dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) -main_node = cluster.add_instance('main_node', main_configs=['configs/dictionaries/cache_ints_dictionary.xml']) +main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', + 'configs/dictionaries/cache_ints_dictionary.xml']) @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py index 7d762db2a6d..d6517379086 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py @@ -9,10 +9,10 @@ from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) +cluster = ClickHouseCluster(__file__) dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) -main_node = cluster.add_instance('main_node', main_configs=['configs/dictionaries/cache_strings_default_settings.xml']) +main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml','configs/dictionaries/cache_ints_dictionary.xml','configs/dictionaries/cache_strings_default_settings.xml']) def get_random_string(string_length=8): @@ -26,7 +26,7 @@ def started_cluster(): dictionary_node.query("CREATE DATABASE IF NOT EXISTS test;") dictionary_node.query("DROP TABLE IF EXISTS test.strings;") dictionary_node.query(""" - CREATE TABLE test.strings + CREATE TABLE test.strings (key UInt64, value String) ENGINE = Memory; """) diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py index 6b0e1936259..44698b380e3 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py @@ -9,11 +9,10 @@ from helpers.cluster import ClickHouseKiller from helpers.network import PartitionManager from helpers.network import PartitionManagerDisabler -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) +cluster = ClickHouseCluster(__file__) dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) -main_node = cluster.add_instance('main_node', main_configs=['configs/dictionaries/cache_ints_dictionary.xml']) +main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', 'configs/dictionaries/cache_ints_dictionary.xml']) @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py index 3fce7b7398d..e0b546aae24 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py @@ -8,11 +8,10 @@ from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseKiller from helpers.network import PartitionManager -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) +cluster = ClickHouseCluster(__file__) dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) -main_node = cluster.add_instance('main_node', main_configs=['configs/dictionaries/cache_ints_dictionary.xml']) +main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml','configs/dictionaries/cache_ints_dictionary.xml']) @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionary_custom_settings/configs/enable_dictionaries.xml b/tests/integration/test_dictionary_custom_settings/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..8a3d6704670 --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/*.xml + diff --git a/tests/integration/test_dictionary_custom_settings/test.py b/tests/integration/test_dictionary_custom_settings/test.py index 97874879525..e58b40df527 100644 --- a/tests/integration/test_dictionary_custom_settings/test.py +++ b/tests/integration/test_dictionary_custom_settings/test.py @@ -3,8 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -config_dir = os.path.join(SCRIPT_DIR, './configs') +ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml'] DICTIONARY_FILES = [ 'configs/dictionaries/FileSourceConfig.xml', 'configs/dictionaries/ExecutableSourceConfig.xml', @@ -13,8 +12,8 @@ DICTIONARY_FILES = [ 'configs/dictionaries/ClickHouseSourceConfig.xml' ] -cluster = ClickHouseCluster(__file__, base_configs_dir=config_dir) -instance = cluster.add_instance('node', main_configs=DICTIONARY_FILES, config_dir=config_dir) +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('node', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES) def prepare(): node = instance diff --git a/tests/integration/test_dictionary_ddl_on_cluster/test.py b/tests/integration/test_dictionary_ddl_on_cluster/test.py index 909d2e06377..6239fda1752 100644 --- a/tests/integration/test_dictionary_ddl_on_cluster/test.py +++ b/tests/integration/test_dictionary_ddl_on_cluster/test.py @@ -4,10 +4,10 @@ from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) -ch1 = cluster.add_instance('ch1', config_dir="configs", with_zookeeper=True) -ch2 = cluster.add_instance('ch2', config_dir="configs", with_zookeeper=True) -ch3 = cluster.add_instance('ch3', config_dir="configs", with_zookeeper=True) -ch4 = cluster.add_instance('ch4', config_dir="configs", with_zookeeper=True) +ch1 = cluster.add_instance('ch1', main_configs=["configs/config.d/clusters.xml", "configs/config.d/ddl.xml"], with_zookeeper=True) +ch2 = cluster.add_instance('ch2', main_configs=["configs/config.d/clusters.xml", "configs/config.d/ddl.xml"], with_zookeeper=True) +ch3 = cluster.add_instance('ch3', main_configs=["configs/config.d/clusters.xml", "configs/config.d/ddl.xml"], with_zookeeper=True) +ch4 = cluster.add_instance('ch4', main_configs=["configs/config.d/clusters.xml", "configs/config.d/ddl.xml"], with_zookeeper=True) @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_disk_types/configs/storage.xml b/tests/integration/test_disk_types/configs/storage.xml new file mode 100644 index 00000000000..2bf9a2e363a --- /dev/null +++ b/tests/integration/test_disk_types/configs/storage.xml @@ -0,0 +1,16 @@ + + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + memory + + + + diff --git a/tests/integration/test_disk_types/test.py b/tests/integration/test_disk_types/test.py index 04346388b47..3c65315a7e3 100644 --- a/tests/integration/test_disk_types/test.py +++ b/tests/integration/test_disk_types/test.py @@ -1,3 +1,4 @@ + import pytest from helpers.cluster import ClickHouseCluster @@ -12,7 +13,7 @@ disk_types = { def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node", config_dir="configs", with_minio=True) + cluster.add_instance("node", main_configs=["configs/storage.xml"], with_minio=True) cluster.start() yield cluster finally: diff --git a/tests/integration/test_distributed_ddl/cluster.py b/tests/integration/test_distributed_ddl/cluster.py index 082a76cd88d..d7cb3d81c82 100644 --- a/tests/integration/test_distributed_ddl/cluster.py +++ b/tests/integration/test_distributed_ddl/cluster.py @@ -17,10 +17,23 @@ class ClickHouseClusterWithDDLHelpers(ClickHouseCluster): def prepare(self, replace_hostnames_with_ips=True): try: + main_configs = [os.path.join(self.test_config_dir, "config.d/clusters.xml"), + os.path.join(self.test_config_dir, "config.d/zookeeper_session_timeout.xml"), + os.path.join(self.test_config_dir, "config.d/macro.xml"), + os.path.join(self.test_config_dir, "config.d/query_log.xml"), + os.path.join(self.test_config_dir, "config.d/ddl.xml")] + user_configs = [os.path.join(self.test_config_dir, "users.d/restricted_user.xml"), + os.path.join(self.test_config_dir, "users.d/query_log.xml")] + if self.test_config_dir == "configs_secure": + main_configs += [os.path.join(self.test_config_dir, "server.crt"), + os.path.join(self.test_config_dir, "server.key"), + os.path.join(self.test_config_dir, "dhparam.pem"), + os.path.join(self.test_config_dir, "config.d/ssl_conf.xml")] for i in xrange(4): self.add_instance( 'ch{}'.format(i+1), - config_dir=self.test_config_dir, + main_configs=main_configs, + user_configs=user_configs, macros={"layer": 0, "shard": i/2 + 1, "replica": i%2 + 1}, with_zookeeper=True) diff --git a/tests/integration/test_distributed_ddl/configs_secure/config.d/ssl_conf.xml b/tests/integration/test_distributed_ddl/configs_secure/config.d/ssl_conf.xml index 696695ddc69..fe39e3712b8 100644 --- a/tests/integration/test_distributed_ddl/configs_secure/config.d/ssl_conf.xml +++ b/tests/integration/test_distributed_ddl/configs_secure/config.d/ssl_conf.xml @@ -1,8 +1,9 @@ - /etc/clickhouse-server/server.crt - /etc/clickhouse-server/server.key + /etc/clickhouse-server/config.d/server.crt + /etc/clickhouse-server/config.d/server.key + /etc/clickhouse-server/config.d/dhparam.pem none true diff --git a/tests/integration/test_distributed_ddl_password/test.py b/tests/integration/test_distributed_ddl_password/test.py index f957f001df1..961b60857dd 100644 --- a/tests/integration/test_distributed_ddl_password/test.py +++ b/tests/integration/test_distributed_ddl_password/test.py @@ -6,12 +6,12 @@ from helpers.test_tools import assert_eq_with_retry from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True) -node2 = cluster.add_instance('node2', config_dir="configs", with_zookeeper=True) -node3 = cluster.add_instance('node3', config_dir="configs", with_zookeeper=True) -node4 = cluster.add_instance('node4', config_dir="configs", with_zookeeper=True) -node5 = cluster.add_instance('node5', config_dir="configs", with_zookeeper=True) -node6 = cluster.add_instance('node6', config_dir="configs", with_zookeeper=True) +node1 = cluster.add_instance('node1', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True) +node3 = cluster.add_instance('node3', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True) +node4 = cluster.add_instance('node4', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True) +node5 = cluster.add_instance('node5', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True) +node6 = cluster.add_instance('node6', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True) @pytest.fixture(scope="module") diff --git a/tests/integration/test_distributed_format/test.py b/tests/integration/test_distributed_format/test.py index 291db89ae4c..251ec766b74 100644 --- a/tests/integration/test_distributed_format/test.py +++ b/tests/integration/test_distributed_format/test.py @@ -9,7 +9,7 @@ from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir="configs", main_configs=['configs/remote_servers.xml']) +node = cluster.add_instance('node', main_configs=['configs/remote_servers.xml']) cluster_param = pytest.mark.parametrize("cluster", [ ('test_cluster'), diff --git a/tests/integration/test_distributed_respect_user_timeouts/configs_secure/config.d/ssl_conf.xml b/tests/integration/test_distributed_respect_user_timeouts/configs_secure/config.d/ssl_conf.xml index 696695ddc69..fe39e3712b8 100644 --- a/tests/integration/test_distributed_respect_user_timeouts/configs_secure/config.d/ssl_conf.xml +++ b/tests/integration/test_distributed_respect_user_timeouts/configs_secure/config.d/ssl_conf.xml @@ -1,8 +1,9 @@ - /etc/clickhouse-server/server.crt - /etc/clickhouse-server/server.key + /etc/clickhouse-server/config.d/server.crt + /etc/clickhouse-server/config.d/server.key + /etc/clickhouse-server/config.d/dhparam.pem none true diff --git a/tests/integration/test_distributed_respect_user_timeouts/test.py b/tests/integration/test_distributed_respect_user_timeouts/test.py index ba760e90412..dc5168bfdad 100644 --- a/tests/integration/test_distributed_respect_user_timeouts/test.py +++ b/tests/integration/test_distributed_respect_user_timeouts/test.py @@ -1,6 +1,6 @@ import itertools import timeit - +import os.path import pytest from helpers.cluster import ClickHouseCluster @@ -91,8 +91,16 @@ def started_cluster(request): cluster = ClickHouseCluster(__file__) cluster.__with_ssl_config = request.param == "configs_secure" + main_configs = [] + main_configs += [os.path.join(request.param, "config.d/remote_servers.xml")] + if cluster.__with_ssl_config: + main_configs += [os.path.join(request.param, "server.crt")] + main_configs += [os.path.join(request.param, "server.key")] + main_configs += [os.path.join(request.param, "dhparam.pem")] + main_configs += [os.path.join(request.param, "config.d/ssl_conf.xml")] + user_configs = [os.path.join(request.param, "users.d/set_distributed_defaults.xml")] for name in NODES: - NODES[name] = cluster.add_instance(name, config_dir=request.param) + NODES[name] = cluster.add_instance(name, main_configs=main_configs, user_configs=user_configs) try: cluster.start() diff --git a/tests/integration/test_distributed_storage_configuration/test.py b/tests/integration/test_distributed_storage_configuration/test.py index 8dfaab659cb..716dd3e3075 100644 --- a/tests/integration/test_distributed_storage_configuration/test.py +++ b/tests/integration/test_distributed_storage_configuration/test.py @@ -9,7 +9,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node', - config_dir='configs', + main_configs=["configs/config.d/storage_configuration.xml"], tmpfs=['/disk1:size=100M', '/disk2:size=100M']) @pytest.fixture(scope='module') diff --git a/tests/integration/test_enabling_access_management/test.py b/tests/integration/test_enabling_access_management/test.py index abb8cd6c07a..4a6ad59f0bb 100644 --- a/tests/integration/test_enabling_access_management/test.py +++ b/tests/integration/test_enabling_access_management/test.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance', config_dir="configs") +instance = cluster.add_instance('instance', user_configs=["configs/users.d/extra_users.xml"]) @pytest.fixture(scope="module", autouse=True) def started_cluster(): diff --git a/tests/integration/test_extreme_deduplication/test.py b/tests/integration/test_extreme_deduplication/test.py index 5c1ae389857..a7e6f10c1f6 100644 --- a/tests/integration/test_extreme_deduplication/test.py +++ b/tests/integration/test_extreme_deduplication/test.py @@ -12,8 +12,8 @@ from helpers.client import QueryTimeoutExceedException cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', config_dir='configs', with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 1}) -node2 = cluster.add_instance('node2', config_dir='configs', with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 2}) +node1 = cluster.add_instance('node1', main_configs=["configs/conf.d/merge_tree.xml", "configs/conf.d/remote_servers.xml"], with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 1}) +node2 = cluster.add_instance('node2', main_configs=["configs/conf.d/merge_tree.xml", "configs/conf.d/remote_servers.xml"], with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 2}) nodes = [node1, node2] @pytest.fixture(scope="module") diff --git a/tests/integration/test_https_replication/configs/dhparam.pem b/tests/integration/test_https_replication/configs/dhparam.pem new file mode 100644 index 00000000000..2e6cee0798d --- /dev/null +++ b/tests/integration/test_https_replication/configs/dhparam.pem @@ -0,0 +1,8 @@ +-----BEGIN DH PARAMETERS----- +MIIBCAKCAQEAua92DDli13gJ+//ZXyGaggjIuidqB0crXfhUlsrBk9BV1hH3i7fR +XGP9rUdk2ubnB3k2ejBStL5oBrkHm9SzUFSQHqfDjLZjKoUpOEmuDc4cHvX1XTR5 +Pr1vf5cd0yEncJWG5W4zyUB8k++SUdL2qaeslSs+f491HBLDYn/h8zCgRbBvxhxb +9qeho1xcbnWeqkN6Kc9bgGozA16P9NLuuLttNnOblkH+lMBf42BSne/TWt3AlGZf +slKmmZcySUhF8aKfJnLKbkBCFqOtFRh8zBA9a7g+BT/lSANATCDPaAk1YVih2EKb +dpc3briTDbRsiqg2JKMI7+VdULY9bh3EawIBAg== +-----END DH PARAMETERS----- diff --git a/tests/integration/test_https_replication/configs/ssl_conf.xml b/tests/integration/test_https_replication/configs/ssl_conf.xml index 237bbc6af1c..ad7b874ebd3 100644 --- a/tests/integration/test_https_replication/configs/ssl_conf.xml +++ b/tests/integration/test_https_replication/configs/ssl_conf.xml @@ -1,8 +1,9 @@ - /etc/clickhouse-server/server.crt - /etc/clickhouse-server/server.key + /etc/clickhouse-server/config.d/server.crt + /etc/clickhouse-server/config.d/server.key + /etc/clickhouse-server/config.d/dhparam.pem none true @@ -15,4 +16,5 @@ 9010 + diff --git a/tests/integration/test_https_replication/test.py b/tests/integration/test_https_replication/test.py index a34c5faeccc..4974da850b4 100644 --- a/tests/integration/test_https_replication/test.py +++ b/tests/integration/test_https_replication/test.py @@ -23,8 +23,8 @@ def _fill_nodes(nodes, shard): '''.format(shard=shard, replica=node.name)) cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml'], with_zookeeper=True) -node2 = cluster.add_instance('node2', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml'], with_zookeeper=True) +node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml', "configs/server.crt", "configs/server.key", "configs/dhparam.pem"], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml', "configs/server.crt", "configs/server.key", "configs/dhparam.pem"], with_zookeeper=True) @pytest.fixture(scope="module") def both_https_cluster(): @@ -78,8 +78,8 @@ def test_replication_after_partition(both_https_cluster): -node3 = cluster.add_instance('node3', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) -node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) +node3 = cluster.add_instance('node3', main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) +node4 = cluster.add_instance('node4', main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) @pytest.fixture(scope="module") def both_http_cluster(): @@ -104,8 +104,8 @@ def test_both_http(both_http_cluster): assert_eq_with_retry(node3, "SELECT id FROM test_table order by id", '111\n222') assert_eq_with_retry(node4, "SELECT id FROM test_table order by id", '111\n222') -node5 = cluster.add_instance('node5', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml'], with_zookeeper=True) -node6 = cluster.add_instance('node6', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) +node5 = cluster.add_instance('node5', main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml', "configs/server.crt", "configs/server.key", "configs/dhparam.pem"], with_zookeeper=True) +node6 = cluster.add_instance('node6', main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) @pytest.fixture(scope="module") def mixed_protocol_cluster(): diff --git a/tests/integration/test_log_family_s3/configs/minio.xml b/tests/integration/test_log_family_s3/configs/minio.xml new file mode 100644 index 00000000000..6c9329a2bbc --- /dev/null +++ b/tests/integration/test_log_family_s3/configs/minio.xml @@ -0,0 +1,13 @@ + + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + + diff --git a/tests/integration/test_log_family_s3/configs/ssl.xml b/tests/integration/test_log_family_s3/configs/ssl.xml new file mode 100644 index 00000000000..95cdc918bd0 --- /dev/null +++ b/tests/integration/test_log_family_s3/configs/ssl.xml @@ -0,0 +1,12 @@ + + + + + true + none + + AcceptCertificateHandler + + + + diff --git a/tests/integration/test_log_family_s3/test.py b/tests/integration/test_log_family_s3/test.py index 50e5b2ad19e..3b0d847967b 100644 --- a/tests/integration/test_log_family_s3/test.py +++ b/tests/integration/test_log_family_s3/test.py @@ -11,7 +11,7 @@ logging.getLogger().addHandler(logging.StreamHandler()) def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node", config_dir="configs", with_minio=True) + cluster.add_instance("node", main_configs=["configs/minio.xml", "configs/ssl.xml", "configs/config.d/log_conf.xml"], with_minio=True) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") diff --git a/tests/integration/test_max_http_connections_for_replication/test.py b/tests/integration/test_max_http_connections_for_replication/test.py index c421d36c315..0317aa19cc3 100644 --- a/tests/integration/test_max_http_connections_for_replication/test.py +++ b/tests/integration/test_max_http_connections_for_replication/test.py @@ -22,8 +22,8 @@ def _fill_nodes(nodes, shard, connections_count): '''.format(shard=shard, replica=node.name, connections=connections_count)) cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) -node2 = cluster.add_instance('node2', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node1 = cluster.add_instance('node1', user_configs=[], main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node2 = cluster.add_instance('node2', user_configs=[], main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) @pytest.fixture(scope="module") def start_small_cluster(): @@ -68,9 +68,9 @@ def test_keepalive_timeout(start_small_cluster): assert not node2.contains_in_log("No message received"), "Found 'No message received' in clickhouse-server.log" -node3 = cluster.add_instance('node3', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) -node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) -node5 = cluster.add_instance('node5', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node3 = cluster.add_instance('node3', user_configs=[], main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node4 = cluster.add_instance('node4', user_configs=[], main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node5 = cluster.add_instance('node5', user_configs=[], main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) @pytest.fixture(scope="module") def start_big_cluster(): diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 01923293b21..de8ec8374e9 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -14,7 +14,7 @@ logging.getLogger().addHandler(logging.StreamHandler()) def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node", config_dir="configs", with_minio=True) + cluster.add_instance("node", main_configs=["configs/config.d/storage_conf.xml", "configs/config.d/bg_processing_pool_conf.xml", "configs/config.d/log_conf.xml"], user_configs=[], with_minio=True) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -55,7 +55,7 @@ def create_table(cluster, table_name, additional_settings=None): ORDER BY (dt, id) SETTINGS storage_policy='s3', - old_parts_lifetime=0, + old_parts_lifetime=0, index_granularity=512 """.format(table_name) diff --git a/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/query_log.xml b/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/query_log.xml new file mode 100644 index 00000000000..afcc8ba5c67 --- /dev/null +++ b/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/query_log.xml @@ -0,0 +1,9 @@ + + + + system + query_log
+ toYYYYMM(event_date) + 1000 +
+
diff --git a/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/ssl_conf.xml b/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/ssl_conf.xml new file mode 100644 index 00000000000..95cdc918bd0 --- /dev/null +++ b/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/ssl_conf.xml @@ -0,0 +1,12 @@ + + + + + true + none + + AcceptCertificateHandler + + + + diff --git a/tests/integration/test_merge_tree_s3_with_cache/test.py b/tests/integration/test_merge_tree_s3_with_cache/test.py index 72c7d97cfed..25c08777ae5 100644 --- a/tests/integration/test_merge_tree_s3_with_cache/test.py +++ b/tests/integration/test_merge_tree_s3_with_cache/test.py @@ -11,7 +11,9 @@ logging.getLogger().addHandler(logging.StreamHandler()) def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node", config_dir="configs", with_minio=True) + cluster.add_instance("node", main_configs=["configs/config.d/log_conf.xml", "configs/config.d/storage_conf.xml", + "configs/config.d/ssl_conf.xml", "configs/config.d/query_log.xml"], + user_configs=["configs/config.d/users.xml"], with_minio=True) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index d00450bf245..9034892ba83 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -13,16 +13,14 @@ from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', - config_dir='configs', - main_configs=['configs/logs_config.xml'], + main_configs=['configs/logs_config.xml', 'configs/config.d/storage_configuration.xml', 'configs/config.d/cluster.xml'], with_zookeeper=True, stay_alive=True, tmpfs=['/jbod1:size=40M', '/jbod2:size=40M', '/external:size=200M'], macros={"shard": 0, "replica": 1} ) node2 = cluster.add_instance('node2', - config_dir='configs', - main_configs=['configs/logs_config.xml'], + main_configs=['configs/logs_config.xml', 'configs/config.d/storage_configuration.xml', 'configs/config.d/cluster.xml'], with_zookeeper=True, stay_alive=True, tmpfs=['/jbod1:size=40M', '/jbod2:size=40M', '/external:size=200M'], diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index 2791cc7b382..efbbe6d4104 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -127,7 +127,6 @@ def test_bad_arguments_for_mysql_database_engine(started_cluster): with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: with pytest.raises(QueryRuntimeException) as exception: mysql_node.query("CREATE DATABASE IF NOT EXISTS test_bad_arguments DEFAULT CHARACTER SET 'utf8'") - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', test_bad_arguments, root, 'clickhouse')") - + clickhouse_node.query("CREATE DATABASE test_database_bad_arguments ENGINE = MySQL('mysql1:3306', test_bad_arguments, root, 'clickhouse')") assert 'Database engine MySQL requested literal argument.' in str(exception.value) mysql_node.query("DROP DATABASE test_bad_arguments") diff --git a/tests/integration/test_mysql_protocol/configs/log_conf.xml b/tests/integration/test_mysql_protocol/configs/log_conf.xml new file mode 100644 index 00000000000..0346e43c81d --- /dev/null +++ b/tests/integration/test_mysql_protocol/configs/log_conf.xml @@ -0,0 +1,10 @@ + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + diff --git a/tests/integration/test_mysql_protocol/configs/mysql.xml b/tests/integration/test_mysql_protocol/configs/mysql.xml new file mode 100644 index 00000000000..a3ebc6e8576 --- /dev/null +++ b/tests/integration/test_mysql_protocol/configs/mysql.xml @@ -0,0 +1,4 @@ + + + 9001 + diff --git a/tests/integration/test_mysql_protocol/configs/ssl_conf.xml b/tests/integration/test_mysql_protocol/configs/ssl_conf.xml new file mode 100644 index 00000000000..5938b80ccb8 --- /dev/null +++ b/tests/integration/test_mysql_protocol/configs/ssl_conf.xml @@ -0,0 +1,18 @@ + + + + + + + /etc/clickhouse-server/config.d/server.crt + /etc/clickhouse-server/config.d/server.key + + /etc/clickhouse-server/config.d/dhparam.pem + none + true + true + sslv2,sslv3 + true + + + diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index a31961dbd16..6e1ef39d2ca 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -17,9 +17,10 @@ from helpers.cluster import ClickHouseCluster, get_docker_compose_path SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) DOCKER_COMPOSE_PATH = get_docker_compose_path() -config_dir = os.path.join(SCRIPT_DIR, './configs') cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir=config_dir, env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) +node = cluster.add_instance('node', main_configs=["configs/log_conf.xml", "configs/ssl_conf.xml", "configs/mysql.xml", + "configs/dhparam.pem", "configs/server.crt", "configs/server.key"], + user_configs=["configs/users.xml"], env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) server_port = 9001 @@ -36,7 +37,7 @@ def server_address(): @pytest.fixture(scope='module') def mysql_client(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_client.yml') - subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) + subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.from_env().containers.get(cluster.project_name + '_mysql1_1') @@ -62,28 +63,28 @@ def mysql_server(mysql_client): @pytest.fixture(scope='module') def golang_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_golang_client.yml') - subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) + subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.from_env().containers.get(cluster.project_name + '_golang1_1') @pytest.fixture(scope='module') def php_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml') - subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) + subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.from_env().containers.get(cluster.project_name + '_php1_1') @pytest.fixture(scope='module') def nodejs_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml') - subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) + subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.from_env().containers.get(cluster.project_name + '_mysqljs1_1') @pytest.fixture(scope='module') def java_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml') - subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) + subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.from_env().containers.get(cluster.project_name + '_java1_1') @@ -329,7 +330,7 @@ def test_python_client(server_address): def test_golang_client(server_address, golang_container): # type: (str, Container) -> None - with open(os.path.join(SCRIPT_DIR,'golang.reference')) as fp: + with open(os.path.join(SCRIPT_DIR, 'golang.reference')) as fp: reference = fp.read() code, (stdout, stderr) = golang_container.exec_run('./main --host {host} --port {port} --user default --password 123 --database ' diff --git a/tests/integration/test_odbc_interaction/configs/enable_dictionaries.xml b/tests/integration/test_odbc_interaction/configs/enable_dictionaries.xml new file mode 100644 index 00000000000..93780125e8e --- /dev/null +++ b/tests/integration/test_odbc_interaction/configs/enable_dictionaries.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/*dictionary.xml + diff --git a/tests/integration/test_odbc_interaction/configs/odbc_logging.xml b/tests/integration/test_odbc_interaction/configs/odbc_logging.xml new file mode 100644 index 00000000000..029275eb09c --- /dev/null +++ b/tests/integration/test_odbc_interaction/configs/odbc_logging.xml @@ -0,0 +1,8 @@ + + + + /var/log/clickhouse-server/clickhouse-odbc-bridge.log + /var/log/clickhouse-server/clickhouse-odbc-bridge.err.log + trace + + diff --git a/tests/integration/test_odbc_interaction/configs/openssl.xml b/tests/integration/test_odbc_interaction/configs/openssl.xml new file mode 100644 index 00000000000..95cdc918bd0 --- /dev/null +++ b/tests/integration/test_odbc_interaction/configs/openssl.xml @@ -0,0 +1,12 @@ + + + + + true + none + + AcceptCertificateHandler + + + + diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 46845802083..33b024363cb 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -7,10 +7,9 @@ import psycopg2 from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT from helpers.cluster import ClickHouseCluster -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) -node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, image='yandex/clickhouse-integration-test', main_configs=['configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml', 'configs/dictionaries/sqlite3_odbc_cached_dictionary.xml', 'configs/dictionaries/postgres_odbc_hashed_dictionary.xml'], stay_alive=True) +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, image='yandex/clickhouse-integration-test', main_configs=['configs/openssl.xml','configs/odbc_logging.xml','configs/enable_dictionaries.xml','configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml','configs/dictionaries/sqlite3_odbc_cached_dictionary.xml','configs/dictionaries/postgres_odbc_hashed_dictionary.xml'], stay_alive=True) create_table_sql_template = """ CREATE TABLE `clickhouse`.`{}` ( diff --git a/tests/integration/test_old_versions/test.py b/tests/integration/test_old_versions/test.py index d77b4af016a..a1770333ba7 100644 --- a/tests/integration/test_old_versions/test.py +++ b/tests/integration/test_old_versions/test.py @@ -1,3 +1,4 @@ + import time import os import pytest @@ -9,13 +10,13 @@ from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) -node18_14 = cluster.add_instance('node18_14', image='yandex/clickhouse-server:18.14.19', with_installed_binary=True, config_dir="configs") -node19_1 = cluster.add_instance('node19_1', image='yandex/clickhouse-server:19.1.16', with_installed_binary=True, config_dir="configs") -node19_4 = cluster.add_instance('node19_4', image='yandex/clickhouse-server:19.4.5.35', with_installed_binary=True, config_dir="configs") -node19_8 = cluster.add_instance('node19_8', image='yandex/clickhouse-server:19.8.3.8', with_installed_binary=True, config_dir="configs") -node19_11 = cluster.add_instance('node19_11', image='yandex/clickhouse-server:19.11.13.74', with_installed_binary=True, config_dir="configs") -node19_13 = cluster.add_instance('node19_13', image='yandex/clickhouse-server:19.13.7.57', with_installed_binary=True, config_dir="configs") -node19_16 = cluster.add_instance('node19_16', image='yandex/clickhouse-server:19.16.2.2', with_installed_binary=True, config_dir="configs") +node18_14 = cluster.add_instance('node18_14', image='yandex/clickhouse-server:18.14.19', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"]) +node19_1 = cluster.add_instance('node19_1', image='yandex/clickhouse-server:19.1.16', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"]) +node19_4 = cluster.add_instance('node19_4', image='yandex/clickhouse-server:19.4.5.35', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"]) +node19_8 = cluster.add_instance('node19_8', image='yandex/clickhouse-server:19.8.3.8', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"]) +node19_11 = cluster.add_instance('node19_11', image='yandex/clickhouse-server:19.11.13.74', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"]) +node19_13 = cluster.add_instance('node19_13', image='yandex/clickhouse-server:19.13.7.57', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"]) +node19_16 = cluster.add_instance('node19_16', image='yandex/clickhouse-server:19.16.2.2', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"]) old_nodes = [node18_14, node19_1, node19_4, node19_8, node19_11, node19_13, node19_16] new_node = cluster.add_instance('node_new') diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index d3ebbd8c7a8..7fd29216680 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -53,21 +53,21 @@ def create_tables_old_format(name, nodes, shard): ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/{name}', '{repl}', date, id, 64) '''.format(name=name, shard=shard, repl=i)) -node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True) -node2 = cluster.add_instance('node2', config_dir="configs", with_zookeeper=True) +node1 = cluster.add_instance('node1', main_configs=[], user_configs=["configs/users.d/not_optimize_count.xml"], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=[], user_configs=["configs/users.d/not_optimize_count.xml"], with_zookeeper=True) settings_default = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0} settings_compact_only = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 1000000, 'min_rows_for_compact_part' : 0} settings_not_adaptive = {'index_granularity_bytes' : 0, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0} -node3 = cluster.add_instance('node3', config_dir="configs", with_zookeeper=True) -node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['configs/no_leader.xml'], with_zookeeper=True) +node3 = cluster.add_instance('node3', main_configs=[], user_configs=["configs/users.d/not_optimize_count.xml"], with_zookeeper=True) +node4 = cluster.add_instance('node4', user_configs=["configs/users.d/not_optimize_count.xml"], main_configs=['configs/no_leader.xml'], with_zookeeper=True) settings_compact = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0} settings_wide = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 0, 'min_rows_for_compact_part' : 0} -node5 = cluster.add_instance('node5', config_dir='configs', main_configs=['configs/compact_parts.xml'], with_zookeeper=True) -node6 = cluster.add_instance('node6', config_dir='configs', main_configs=['configs/compact_parts.xml'], with_zookeeper=True) +node5 = cluster.add_instance('node5', main_configs=['configs/compact_parts.xml'], with_zookeeper=True) +node6 = cluster.add_instance('node6', main_configs=['configs/compact_parts.xml'], with_zookeeper=True) settings_in_memory = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 256} @@ -213,8 +213,8 @@ def test_different_part_types_on_replicas(start_cluster, table, part_type): "WHERE table = '{}' AND active GROUP BY part_type ORDER BY part_type".format(table))) == TSV(expected) -node7 = cluster.add_instance('node7', config_dir="configs_old", with_zookeeper=True, image='yandex/clickhouse-server:19.17.8.54', stay_alive=True, with_installed_binary=True) -node8 = cluster.add_instance('node8', config_dir="configs", with_zookeeper=True) +node7 = cluster.add_instance('node7', user_configs=["configs_old/users.d/not_optimize_count.xml"], with_zookeeper=True, image='yandex/clickhouse-server:19.17.8.54', stay_alive=True, with_installed_binary=True) +node8 = cluster.add_instance('node8', main_configs=[], user_configs=["configs/users.d/not_optimize_count.xml"], with_zookeeper=True) settings7 = {'index_granularity_bytes' : 10485760} settings8 = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0} diff --git a/tests/integration/test_postgresql_protocol/configs/default_passwd.xml b/tests/integration/test_postgresql_protocol/configs/default_passwd.xml new file mode 100644 index 00000000000..86f5b6657c2 --- /dev/null +++ b/tests/integration/test_postgresql_protocol/configs/default_passwd.xml @@ -0,0 +1,13 @@ + + + + + + + + + + 123 + + + diff --git a/tests/integration/test_postgresql_protocol/configs/log.xml b/tests/integration/test_postgresql_protocol/configs/log.xml new file mode 100644 index 00000000000..7f6380b0393 --- /dev/null +++ b/tests/integration/test_postgresql_protocol/configs/log.xml @@ -0,0 +1,10 @@ + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + diff --git a/tests/integration/test_postgresql_protocol/configs/postresql.xml b/tests/integration/test_postgresql_protocol/configs/postresql.xml new file mode 100644 index 00000000000..aedfb59bedb --- /dev/null +++ b/tests/integration/test_postgresql_protocol/configs/postresql.xml @@ -0,0 +1,4 @@ + + + 5433 + diff --git a/tests/integration/test_postgresql_protocol/configs/ssl_conf.xml b/tests/integration/test_postgresql_protocol/configs/ssl_conf.xml new file mode 100644 index 00000000000..271cb987218 --- /dev/null +++ b/tests/integration/test_postgresql_protocol/configs/ssl_conf.xml @@ -0,0 +1,18 @@ + + + + + + + /etc/clickhouse-server/config.d/server.crt + /etc/clickhouse-server/config.d/server.key + + /etc/clickhouse-server/config.d/dhparam.pem + none + true + true + sslv2,sslv3 + true + + + diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index 527c652229e..939e8231931 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -19,11 +19,12 @@ from helpers.cluster import ClickHouseCluster, get_docker_compose_path psycopg2.extras.register_uuid() SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -config_dir = os.path.join(SCRIPT_DIR, './configs') DOCKER_COMPOSE_PATH = get_docker_compose_path() cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir=config_dir, env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) +node = cluster.add_instance('node', main_configs=["configs/postresql.xml", "configs/log.xml", "configs/ssl_conf.xml", + "configs/dhparam.pem", "configs/server.crt", "configs/server.key"], + user_configs=["configs/default_passwd.xml"], env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) server_port = 5433 diff --git a/tests/integration/test_profile_events_s3/configs/log.xml b/tests/integration/test_profile_events_s3/configs/log.xml new file mode 100644 index 00000000000..0346e43c81d --- /dev/null +++ b/tests/integration/test_profile_events_s3/configs/log.xml @@ -0,0 +1,10 @@ + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + diff --git a/tests/integration/test_profile_events_s3/configs/query_log.xml b/tests/integration/test_profile_events_s3/configs/query_log.xml new file mode 100644 index 00000000000..afcc8ba5c67 --- /dev/null +++ b/tests/integration/test_profile_events_s3/configs/query_log.xml @@ -0,0 +1,9 @@ + + + + system + query_log
+ toYYYYMM(event_date) + 1000 +
+
diff --git a/tests/integration/test_profile_events_s3/configs/ssl_conf.xml b/tests/integration/test_profile_events_s3/configs/ssl_conf.xml new file mode 100644 index 00000000000..95cdc918bd0 --- /dev/null +++ b/tests/integration/test_profile_events_s3/configs/ssl_conf.xml @@ -0,0 +1,12 @@ + + + + + true + none + + AcceptCertificateHandler + + + + diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index f98505757bf..e2cb10499e7 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -17,7 +17,7 @@ def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node", config_dir="configs", with_minio=True) + cluster.add_instance("node", main_configs=["configs/config.d/storage_conf.xml", "configs/log.xml", "configs/query_log.xml", "configs/ssl_conf.xml"], with_minio=True) logging.info("Starting cluster...") cluster.start() diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index f490c13ca27..e89611c0d99 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -7,18 +7,15 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -zero = cluster.add_instance("zero", - config_dir="configs", +zero = cluster.add_instance("zero", user_configs=["configs/users.d/settings.xml"], macros={"cluster": "anime", "shard": "0", "replica": "zero"}, with_zookeeper=True) -first = cluster.add_instance("first", - config_dir="configs", +first = cluster.add_instance("first", user_configs=["configs/users.d/settings.xml"], macros={"cluster": "anime", "shard": "0", "replica": "first"}, with_zookeeper=True) -second = cluster.add_instance("second", - config_dir="configs", +second = cluster.add_instance("second", user_configs=["configs/users.d/settings.xml"], macros={"cluster": "anime", "shard": "0", "replica": "second"}, with_zookeeper=True) diff --git a/tests/integration/test_quota/test.py b/tests/integration/test_quota/test.py index 27aa353b9b1..4c97d127ad0 100644 --- a/tests/integration/test_quota/test.py +++ b/tests/integration/test_quota/test.py @@ -6,28 +6,38 @@ import re import time cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance', - config_dir="configs") +instance = cluster.add_instance('instance', user_configs=["configs/users.d/assign_myquota.xml", "configs/users.d/drop_default_quota.xml", "configs/users.d/quota.xml"]) -def system_quotas(): - return TSV(instance.query("SELECT * FROM system.quotas ORDER BY name")) +def check_system_quotas(canonical): + canonical_tsv = TSV(canonical) + r = TSV(instance.query("SELECT * FROM system.quotas ORDER BY name")) + print("system_quotas: {},\ncanonical: {}".format(r, TSV(canonical_tsv))) + assert r == canonical_tsv -def system_quota_limits(): - return TSV(instance.query("SELECT * FROM system.quota_limits ORDER BY quota_name, duration")) +def system_quota_limits(canonical): + canonical_tsv = TSV(canonical) + r = TSV(instance.query("SELECT * FROM system.quota_limits ORDER BY quota_name, duration")) + print("system_quota_limits: {},\ncanonical: {}".format(r, TSV(canonical_tsv))) + assert r == canonical_tsv -def system_quota_usage(): +def system_quota_usage(canonical): + canonical_tsv = TSV(canonical) query = "SELECT quota_name, quota_key, duration, queries, max_queries, errors, max_errors, result_rows, max_result_rows,"\ "result_bytes, max_result_bytes, read_rows, max_read_rows, read_bytes, max_read_bytes, max_execution_time "\ "FROM system.quota_usage ORDER BY duration" - return TSV(instance.query(query)) + r = TSV(instance.query(query)) + print("system_quota_usage: {},\ncanonical: {}".format(r, TSV(canonical_tsv))) + assert r == canonical_tsv -def system_quotas_usage(): +def system_quotas_usage(canonical): + canonical_tsv = TSV(canonical) query = "SELECT quota_name, quota_key, is_current, duration, queries, max_queries, errors, max_errors, result_rows, max_result_rows, "\ "result_bytes, max_result_bytes, read_rows, max_read_rows, read_bytes, max_read_bytes, max_execution_time "\ "FROM system.quotas_usage ORDER BY quota_name, quota_key, duration" - return TSV(instance.query(query)) - + r = TSV(instance.query(query)) + print("system_quotas_usage: {},\ncanonical: {}".format(r, TSV(canonical_tsv))) + assert r == canonical_tsv def copy_quota_xml(local_file_name, reload_immediately = True): script_dir = os.path.dirname(os.path.realpath(__file__)) @@ -40,7 +50,7 @@ def copy_quota_xml(local_file_name, reload_immediately = True): def started_cluster(): try: cluster.start() - + instance.query("CREATE TABLE test_table(x UInt32) ENGINE = MergeTree ORDER BY tuple()") instance.query("INSERT INTO test_table SELECT number FROM numbers(50)") @@ -61,141 +71,141 @@ def reset_quotas_and_usage_info(): def test_quota_from_users_xml(): - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] - assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] - assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) + system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) instance.query("SELECT * from test_table") - assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"]] + system_quota_usage([["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"]]) instance.query("SELECT COUNT() from test_table") - assert system_quota_usage() == [["myQuota", "default", 31556952, 2, 1000, 0, "\N", 51, "\N", 208, "\N", 50, 1000, 200, "\N", "\N"]] + system_quota_usage([["myQuota", "default", 31556952, 2, 1000, 0, "\N", 51, "\N", 208, "\N", 50, 1000, 200, "\N", "\N"]]) def test_simpliest_quota(): # Simpliest quota doesn't even track usage. copy_quota_xml('simpliest.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]] - assert system_quota_limits() == "" - assert system_quota_usage() == [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]]) + system_quota_limits("") + system_quota_usage([["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]) instance.query("SELECT * from test_table") - assert system_quota_usage() == [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]] + system_quota_usage([["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]) def test_tracking_quota(): # Now we're tracking usage. copy_quota_xml('tracking.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, "\N", "\N", "\N", "\N", "\N", "\N", "\N"]] - assert system_quota_usage() == [["myQuota", "default", 31556952, 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", "\N"]]) instance.query("SELECT * from test_table") - assert system_quota_usage() == [["myQuota", "default", 31556952, 1, "\N", 0, "\N", 50, "\N", 200, "\N", 50, "\N", 200, "\N", "\N"]] + system_quota_usage([["myQuota", "default", 31556952, 1, "\N", 0, "\N", 50, "\N", 200, "\N", 50, "\N", 200, "\N", "\N"]]) instance.query("SELECT COUNT() from test_table") - assert system_quota_usage() == [["myQuota", "default", 31556952, 2, "\N", 0, "\N", 51, "\N", 208, "\N", 50, "\N", 200, "\N", "\N"]] + system_quota_usage([["myQuota", "default", 31556952, 2, "\N", 0, "\N", 51, "\N", 208, "\N", 50, "\N", 200, "\N", "\N"]]) def test_exceed_quota(): # Change quota, now the limits are tiny so we will exceed the quota. copy_quota_xml('tiny_limits.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1, 1, 1, "\N", 1, "\N", "\N"]] - assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1, 0, 1, 0, 1, 0, "\N", 0, 1, 0, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1, 1, 1, "\N", 1, "\N", "\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 0, 1, 0, 1, 0, 1, 0, "\N", 0, 1, 0, "\N", "\N"]]) assert re.search("Quota.*has\ been\ exceeded", instance.query_and_get_error("SELECT * from test_table")) - assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1, 1, 1, 0, 1, 0, "\N", 50, 1, 0, "\N", "\N"]] + system_quota_usage([["myQuota", "default", 31556952, 1, 1, 1, 1, 0, 1, 0, "\N", 50, 1, 0, "\N", "\N"]]) # Change quota, now the limits are enough to execute queries. copy_quota_xml('normal_limits.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] - assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1000, 1, "\N", 0, "\N", 0, "\N", 50, 1000, 0, "\N", "\N"]] - + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 1, 1000, 1, "\N", 0, "\N", 0, "\N", 50, 1000, 0, "\N", "\N"]]) + instance.query("SELECT * from test_table") - assert system_quota_usage() == [["myQuota", "default", 31556952, 2, 1000, 1, "\N", 50, "\N", 200, "\N", 100, 1000, 200, "\N", "\N"]] + system_quota_usage([["myQuota", "default", 31556952, 2, 1000, 1, "\N", 50, "\N", 200, "\N", 100, 1000, 200, "\N", "\N"]]) def test_add_remove_interval(): - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] - assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) # Add interval. copy_quota_xml('two_intervals.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952,63113904]", 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"], - ["myQuota", 63113904, 1, "\N", "\N", "\N", 30000, "\N", 20000, 120]] - assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"], - ["myQuota", "default", 63113904, 0, "\N", 0, "\N", 0, "\N", 0, 30000, 0, "\N", 0, 20000, 120]] - + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952,63113904]", 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"], + ["myQuota", 63113904, 1, "\N", "\N", "\N", 30000, "\N", 20000, 120]]) + system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"], + ["myQuota", "default", 63113904, 0, "\N", 0, "\N", 0, "\N", 0, 30000, 0, "\N", 0, 20000, 120]]) + instance.query("SELECT * from test_table") - assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"], - ["myQuota", "default", 63113904, 1, "\N", 0, "\N", 50, "\N", 200, 30000, 50, "\N", 200, 20000, 120]] + system_quota_usage([["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"], + ["myQuota", "default", 63113904, 1, "\N", 0, "\N", 50, "\N", 200, 30000, 50, "\N", 200, 20000, 120]]) # Remove interval. copy_quota_xml('normal_limits.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] - assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"]] - + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"]]) + instance.query("SELECT * from test_table") - assert system_quota_usage() == [["myQuota", "default", 31556952, 2, 1000, 0, "\N", 100, "\N", 400, "\N", 100, 1000, 400, "\N", "\N"]] + system_quota_usage([["myQuota", "default", 31556952, 2, 1000, 0, "\N", 100, "\N", 400, "\N", 100, 1000, 400, "\N", "\N"]]) # Remove all intervals. copy_quota_xml('simpliest.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]] - assert system_quota_limits() == "" - assert system_quota_usage() == [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]] - + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]]) + system_quota_limits("") + system_quota_usage([["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]) + instance.query("SELECT * from test_table") - assert system_quota_usage() == [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]] + system_quota_usage([["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]) # Add one interval back. copy_quota_xml('normal_limits.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] - assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) def test_add_remove_quota(): - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] - assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) # Add quota. copy_quota_xml('two_quotas.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"], - ["myQuota2", "4590510c-4d13-bf21-ec8a-c2187b092e73", "users.xml", "['client_key','user_name']", "[3600,2629746]", 0, "[]", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"], + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"], + ["myQuota2", "4590510c-4d13-bf21-ec8a-c2187b092e73", "users.xml", "['client_key','user_name']", "[3600,2629746]", 0, "[]", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"], ["myQuota2", 3600, 1, "\N", "\N", 4000, 400000, 4000, 400000, 60], - ["myQuota2", 2629746, 0, "\N", "\N", "\N", "\N", "\N", "\N", 1800]] - assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] + ["myQuota2", 2629746, 0, "\N", "\N", "\N", "\N", "\N", "\N", 1800]]) + system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) # Drop quota. copy_quota_xml('normal_limits.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] - assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) # Drop all quotas. copy_quota_xml('no_quotas.xml') - assert system_quotas() == "" - assert system_quota_limits() == "" - assert system_quotas_usage() == "" + check_system_quotas("") + system_quota_limits("") + system_quotas_usage("") # Add one quota back. copy_quota_xml('normal_limits.xml') - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] - assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) def test_reload_users_xml_by_timer(): - assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] - assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] + check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) time.sleep(1) # The modification time of the 'quota.xml' file should be different, # because config files are reload by timer only when the modification time is changed. @@ -246,7 +256,7 @@ def test_dcl_introspection(): def test_dcl_management(): copy_quota_xml('no_quotas.xml') assert instance.query("SHOW QUOTA") == "" - + instance.query("CREATE QUOTA qA FOR INTERVAL 15 MONTH MAX QUERIES 123 TO CURRENT_USER") assert instance.query("SHOW CREATE QUOTA qA") == "CREATE QUOTA qA FOR INTERVAL 5 quarter MAX queries = 123 TO default\n" assert re.match("qA\\t\\t.*\\t39446190\\t0\\t123\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t.*\\t\\\\N\n", diff --git a/tests/integration/test_random_inserts/test.py b/tests/integration/test_random_inserts/test.py index eb644a7a19c..4e3d8db7e53 100644 --- a/tests/integration/test_random_inserts/test.py +++ b/tests/integration/test_random_inserts/test.py @@ -14,8 +14,8 @@ from helpers.client import CommandRequest cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', config_dir='configs', with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 1}) -node2 = cluster.add_instance('node2', config_dir='configs', with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 2}) +node1 = cluster.add_instance('node1', main_configs=["configs/conf.d/merge_tree.xml", "configs/conf.d/remote_servers.xml" ], with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 1}) +node2 = cluster.add_instance('node2', main_configs=["configs/conf.d/merge_tree.xml", "configs/conf.d/remote_servers.xml" ], with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 2}) nodes = [node1, node2] @pytest.fixture(scope="module") diff --git a/tests/integration/test_reload_max_table_size_to_drop/configs/max_table_size_to_drop.xml b/tests/integration/test_reload_max_table_size_to_drop/configs/max_table_size_to_drop.xml new file mode 100644 index 00000000000..03d5e33646f --- /dev/null +++ b/tests/integration/test_reload_max_table_size_to_drop/configs/max_table_size_to_drop.xml @@ -0,0 +1,5 @@ + + + 1 + 1 + diff --git a/tests/integration/test_reload_max_table_size_to_drop/test.py b/tests/integration/test_reload_max_table_size_to_drop/test.py index 3959b383fc5..9d0bc244521 100644 --- a/tests/integration/test_reload_max_table_size_to_drop/test.py +++ b/tests/integration/test_reload_max_table_size_to_drop/test.py @@ -1,3 +1,4 @@ + import time import pytest import os @@ -6,10 +7,10 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir="configs") +node = cluster.add_instance('node', main_configs=["configs/max_table_size_to_drop.xml"]) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -CONFIG_PATH = os.path.join(SCRIPT_DIR, './_instances/node/configs/config.xml') +CONFIG_PATH = os.path.join(SCRIPT_DIR, './_instances/node/configs/config.d/max_table_size_to_drop.xml') @pytest.fixture(scope="module") diff --git a/tests/integration/test_rename_column/test.py b/tests/integration/test_rename_column/test.py index 029d140d0ed..9a108583347 100644 --- a/tests/integration/test_rename_column/test.py +++ b/tests/integration/test_rename_column/test.py @@ -12,8 +12,9 @@ from helpers.test_tools import TSV node_options = dict( with_zookeeper=True, - main_configs=['configs/remote_servers.xml'], - config_dir='configs', + main_configs=["configs/remote_servers.xml", "configs/config.d/instant_moves.xml", + "configs/config.d/part_log.xml", "configs/config.d/zookeeper_session_timeout.xml", + "configs/config.d/storage_configuration.xml"], tmpfs=['/external:size=200M', '/internal:size=1M']) cluster = ClickHouseCluster(__file__) diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index 69d41b1ce11..9e617506d29 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -15,9 +15,9 @@ def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node1", config_dir="configs", macros={'cluster': 'test1'}, with_minio=True, with_zookeeper=True) - cluster.add_instance("node2", config_dir="configs", macros={'cluster': 'test1'}, with_zookeeper=True) - cluster.add_instance("node3", config_dir="configs", macros={'cluster': 'test1'}, with_zookeeper=True) + cluster.add_instance("node1", main_configs=["configs/config.d/storage_conf.xml"], macros={'cluster': 'test1'}, with_minio=True, with_zookeeper=True) + cluster.add_instance("node2", main_configs=["configs/config.d/storage_conf.xml"], macros={'cluster': 'test1'}, with_zookeeper=True) + cluster.add_instance("node3", main_configs=["configs/config.d/storage_conf.xml"], macros={'cluster': 'test1'}, with_zookeeper=True) logging.info("Starting cluster...") cluster.start() diff --git a/tests/integration/test_row_policy/configs/users.d/another_user.xml b/tests/integration/test_row_policy/configs/users.d/another_user.xml new file mode 100644 index 00000000000..fb9608e5313 --- /dev/null +++ b/tests/integration/test_row_policy/configs/users.d/another_user.xml @@ -0,0 +1,13 @@ + + + + + + + ::/0 + + default + default + + + \ No newline at end of file diff --git a/tests/integration/test_row_policy/configs/users.d/any_join_distinct_right_table_keys.xml b/tests/integration/test_row_policy/configs/users.d/any_join_distinct_right_table_keys.xml new file mode 100644 index 00000000000..413e64ba3dc --- /dev/null +++ b/tests/integration/test_row_policy/configs/users.d/any_join_distinct_right_table_keys.xml @@ -0,0 +1,8 @@ + + + + + 1 + + + diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index 15796ff0c83..dd0495df237 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -6,8 +6,8 @@ import re import time cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir="configs", with_zookeeper=True) -node2 = cluster.add_instance('node2', config_dir="configs", with_zookeeper=True) +node = cluster.add_instance('node', main_configs=["configs/config.d/remote_servers.xml"], user_configs=["configs/users.d/row_policy.xml", "configs/users.d/another_user.xml", "configs/users.d/any_join_distinct_right_table_keys.xml"], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=["configs/config.d/remote_servers.xml"], user_configs=["configs/users.d/row_policy.xml", "configs/users.d/another_user.xml", "configs/users.d/any_join_distinct_right_table_keys.xml"], with_zookeeper=True) nodes = [node, node2] @@ -42,7 +42,7 @@ def started_cluster(): CREATE TABLE mydb.`.filtered_table4` (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a; INSERT INTO mydb.`.filtered_table4` values (0, 0), (0, 1), (1, 0), (1, 1); - + CREATE TABLE mydb.local (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a; ''') @@ -185,7 +185,7 @@ def test_introspection(): def test_dcl_introspection(): assert node.query("SHOW POLICIES") == TSV(["another ON mydb.filtered_table1", "another ON mydb.filtered_table2", "another ON mydb.filtered_table3", "another ON mydb.local", "default ON mydb.filtered_table1", "default ON mydb.filtered_table2", "default ON mydb.filtered_table3", "default ON mydb.local"]) - + assert node.query("SHOW POLICIES ON mydb.filtered_table1") == TSV([ "another", "default" ]) assert node.query("SHOW POLICIES ON mydb.local") == TSV([ "another", "default" ]) assert node.query("SHOW POLICIES ON mydb.*") == TSV([ "another ON mydb.filtered_table1", "another ON mydb.filtered_table2", "another ON mydb.filtered_table3", "another ON mydb.local", "default ON mydb.filtered_table1", "default ON mydb.filtered_table2", "default ON mydb.filtered_table3", "default ON mydb.local" ]) @@ -195,7 +195,7 @@ def test_dcl_introspection(): assert node.query("SHOW CREATE POLICY default ON mydb.filtered_table2") == "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default\n" assert node.query("SHOW CREATE POLICY default ON mydb.filtered_table3") == "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default\n" assert node.query("SHOW CREATE POLICY default ON mydb.local") == "CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default\n" - + assert node.query("SHOW CREATE POLICY default") == TSV([ "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default", "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default", "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default", "CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default" ]) assert node.query("SHOW CREATE POLICIES ON mydb.filtered_table1") == TSV([ "CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another", "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default" ]) assert node.query("SHOW CREATE POLICIES ON mydb.*") == TSV([ "CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another", "CREATE ROW POLICY another ON mydb.filtered_table2 FOR SELECT USING 1 TO another", "CREATE ROW POLICY another ON mydb.filtered_table3 FOR SELECT USING 1 TO another", "CREATE ROW POLICY another ON mydb.local FOR SELECT USING a = 1 TO another", "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default", "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default", "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default", "CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default" ]) diff --git a/tests/integration/test_s3_with_https/configs/config.d/ssl.xml b/tests/integration/test_s3_with_https/configs/config.d/ssl.xml new file mode 100644 index 00000000000..95cdc918bd0 --- /dev/null +++ b/tests/integration/test_s3_with_https/configs/config.d/ssl.xml @@ -0,0 +1,12 @@ + + + + + true + none + + AcceptCertificateHandler + + + + diff --git a/tests/integration/test_s3_with_https/test.py b/tests/integration/test_s3_with_https/test.py index 81e57106afc..2b40e02e701 100644 --- a/tests/integration/test_s3_with_https/test.py +++ b/tests/integration/test_s3_with_https/test.py @@ -18,7 +18,7 @@ def check_proxy_logs(cluster, proxy_instance): def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node", config_dir="configs", with_minio=True, minio_certs_dir='minio_certs') + cluster.add_instance("node", main_configs=["configs/config.d/storage_conf.xml", "configs/config.d/log_conf.xml", "configs/config.d/ssl.xml"], with_minio=True, minio_certs_dir='minio_certs') logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 0642cd88fe7..daf53c2e27b 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -21,7 +21,7 @@ def run_resolver(cluster): def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node", config_dir="configs", with_minio=True) + cluster.add_instance("node", main_configs=["configs/config.d/log_conf.xml", "configs/config.d/storage_conf.xml"], with_minio=True) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") diff --git a/tests/integration/test_settings_constraints/test.py b/tests/integration/test_settings_constraints/test.py index 1c8e91484ca..b2dcd80448f 100644 --- a/tests/integration/test_settings_constraints/test.py +++ b/tests/integration/test_settings_constraints/test.py @@ -2,8 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance', - config_dir="configs") +instance = cluster.add_instance('instance', user_configs=["configs/users.xml"]) diff --git a/tests/integration/test_settings_constraints_distributed/test.py b/tests/integration/test_settings_constraints_distributed/test.py index 7f0f8868bcf..94afa0d6d2d 100644 --- a/tests/integration/test_settings_constraints_distributed/test.py +++ b/tests/integration/test_settings_constraints_distributed/test.py @@ -8,9 +8,9 @@ from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', config_dir="configs") -node2 = cluster.add_instance('node2', config_dir="configs") -distributed = cluster.add_instance('distributed', config_dir="configs", stay_alive=True) +node1 = cluster.add_instance('node1', main_configs=["configs/config.d/remote_servers.xml"], user_configs=["configs/users.d/allow_introspection_functions.xml"]) +node2 = cluster.add_instance('node2', main_configs=["configs/config.d/remote_servers.xml"], user_configs=["configs/users.d/allow_introspection_functions.xml"]) +distributed = cluster.add_instance('distributed', main_configs=["configs/config.d/remote_servers.xml"], user_configs=["configs/users.d/allow_introspection_functions.xml"], stay_alive=True) @pytest.fixture(scope="module", autouse=True) @@ -56,7 +56,7 @@ def test_select_clamps_settings(): assert distributed.query(query, user = 'normal') == '2\n' assert distributed.query(query, user = 'wasteful') == '2\n' assert distributed.query(query, user = 'readonly') == '2\n' - + assert distributed.query(query, settings={"max_memory_usage": 40000000, "readonly": 2}) == '2\n' assert distributed.query(query, settings={"max_memory_usage": 3000000000, "readonly": 2}) == '2\n' diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index d65b0efc334..20613bde1bc 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -12,7 +12,7 @@ import subprocess SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', with_hdfs=True, config_dir="configs", main_configs=['configs/log_conf.xml']) +node1 = cluster.add_instance('node1', with_hdfs=True, user_configs=[], main_configs=['configs/log_conf.xml']) @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 42b7101f9c6..5ebde084de7 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -20,7 +20,6 @@ from google.protobuf.internal.encoder import _VarintBytes cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', - config_dir='configs', main_configs=['configs/rabbitmq.xml','configs/log_conf.xml'], with_rabbitmq=True) rabbitmq_id = '' diff --git a/tests/integration/test_system_queries/configs/dictionaries/dictionary_clickhouse_cache.xml b/tests/integration/test_system_queries/configs/dictionaries/dictionary_clickhouse_cache.xml index a149c2ba774..806a59debca 100644 --- a/tests/integration/test_system_queries/configs/dictionaries/dictionary_clickhouse_cache.xml +++ b/tests/integration/test_system_queries/configs/dictionaries/dictionary_clickhouse_cache.xml @@ -1,4 +1,4 @@ - + clickhouse_cache @@ -34,4 +34,4 @@ - +
\ No newline at end of file diff --git a/tests/integration/test_system_queries/configs/dictionaries/dictionary_clickhouse_flat.xml b/tests/integration/test_system_queries/configs/dictionaries/dictionary_clickhouse_flat.xml index feb01b27d1b..e7d32590a39 100644 --- a/tests/integration/test_system_queries/configs/dictionaries/dictionary_clickhouse_flat.xml +++ b/tests/integration/test_system_queries/configs/dictionaries/dictionary_clickhouse_flat.xml @@ -1,4 +1,4 @@ - + clickhouse_flat @@ -34,4 +34,4 @@ - +
\ No newline at end of file diff --git a/tests/integration/test_system_queries/test.py b/tests/integration/test_system_queries/test.py index 6f36a13b184..db9cf5ccf3c 100644 --- a/tests/integration/test_system_queries/test.py +++ b/tests/integration/test_system_queries/test.py @@ -18,13 +18,14 @@ def started_cluster(): global instance try: cluster = ClickHouseCluster(__file__) - cluster.add_instance('ch1', config_dir="configs") + cluster.add_instance('ch1', main_configs=["configs/config.d/clusters_config.xml", "configs/config.d/query_log.xml"], + dictionaries=["configs/dictionaries/dictionary_clickhouse_cache.xml", "configs/dictionaries/dictionary_clickhouse_flat.xml"]) cluster.start() instance = cluster.instances['ch1'] instance.query('CREATE DATABASE dictionaries ENGINE = Dictionary') instance.query('CREATE TABLE dictionary_source (id UInt64, value UInt8) ENGINE = Memory') - #print instance.query('SELECT * FROM system.dictionaries FORMAT Vertical') + print instance.query('SELECT * FROM system.dictionaries FORMAT Vertical') print "Started ", instance.ip_address yield cluster @@ -90,7 +91,7 @@ def test_RELOAD_CONFIG_AND_MACROS(started_cluster): instance.exec_in_container(['bash', '-c', create_macros], privileged=True, user='root') instance.query("SYSTEM RELOAD CONFIG") - assert TSV(instance.query("select * from system.macros")) == TSV("mac\tro\n") + assert TSV(instance.query("select * from system.macros")) == TSV("instance\tch1\nmac\tro\n") def test_SYSTEM_FLUSH_LOGS(started_cluster): diff --git a/tests/integration/test_text_log_level/test.py b/tests/integration/test_text_log_level/test.py index d7cf72fd9ea..799ae9021cb 100644 --- a/tests/integration/test_text_log_level/test.py +++ b/tests/integration/test_text_log_level/test.py @@ -8,7 +8,7 @@ from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir='configs') +node = cluster.add_instance('node', main_configs=["configs/config.d/text_log.xml"]) @pytest.fixture(scope='module') def start_cluster(): diff --git a/tests/integration/test_tmp_policy/test.py b/tests/integration/test_tmp_policy/test.py index 5c5900cc9dc..728c62d82fb 100644 --- a/tests/integration/test_tmp_policy/test.py +++ b/tests/integration/test_tmp_policy/test.py @@ -8,7 +8,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node', - config_dir='configs', + main_configs=["configs/config.d/storage_configuration.xml"], tmpfs=['/disk1:size=100M', '/disk2:size=100M']) @pytest.fixture(scope='module') diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index eedcb01ee3a..d0db52287ca 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -14,15 +14,13 @@ from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', - config_dir='configs', - main_configs=['configs/logs_config.xml'], + main_configs=['configs/logs_config.xml', "configs/config.d/instant_moves.xml", "configs/config.d/storage_configuration.xml", "configs/config.d/cluster.xml",], with_zookeeper=True, tmpfs=['/jbod1:size=40M', '/jbod2:size=40M', '/external:size=200M'], macros={"shard": 0, "replica": 1} ) node2 = cluster.add_instance('node2', - config_dir='configs', - main_configs=['configs/logs_config.xml'], + main_configs=['configs/logs_config.xml', "configs/config.d/instant_moves.xml", "configs/config.d/storage_configuration.xml", "configs/config.d/cluster.xml",], with_zookeeper=True, tmpfs=['/jbod1:size=40M', '/jbod2:size=40M', '/external:size=200M'], macros={"shard": 0, "replica": 2} ) @@ -173,7 +171,7 @@ def test_moves_work_after_storage_policy_change(started_cluster, name, engine): ) ENGINE = {engine} ORDER BY tuple() """.format(name=name, engine=engine)) - + node1.query("""ALTER TABLE {name} MODIFY SETTING storage_policy='default_with_small_jbod_with_external'""".format(name=name)) # Second expression is preferred because d1 > now()-3600. diff --git a/tests/integration/test_user_ip_restrictions/test.py b/tests/integration/test_user_ip_restrictions/test.py index 731f2bd7fa8..aee0819fe95 100644 --- a/tests/integration/test_user_ip_restrictions/test.py +++ b/tests/integration/test_user_ip_restrictions/test.py @@ -7,16 +7,16 @@ from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) -node_ipv4 = cluster.add_instance('node_ipv4', config_dir="configs", user_configs=['configs/users_ipv4.xml'], ipv4_address='10.5.172.77') -client_ipv4_ok = cluster.add_instance('client_ipv4_ok', config_dir="configs", ipv4_address='10.5.172.10') -client_ipv4_ok_direct = cluster.add_instance('client_ipv4_ok_direct', config_dir="configs", ipv4_address='10.5.173.1') -client_ipv4_ok_full_mask = cluster.add_instance('client_ipv4_ok_full_mask', config_dir="configs", ipv4_address='10.5.175.77') -client_ipv4_bad = cluster.add_instance('client_ipv4_bad', config_dir="configs", ipv4_address='10.5.173.10') +node_ipv4 = cluster.add_instance('node_ipv4', main_configs=[], user_configs=['configs/users_ipv4.xml'], ipv4_address='10.5.172.77') +client_ipv4_ok = cluster.add_instance('client_ipv4_ok', main_configs=[], user_configs=[], ipv4_address='10.5.172.10') +client_ipv4_ok_direct = cluster.add_instance('client_ipv4_ok_direct', main_configs=[], user_configs=[], ipv4_address='10.5.173.1') +client_ipv4_ok_full_mask = cluster.add_instance('client_ipv4_ok_full_mask', main_configs=[], user_configs=[], ipv4_address='10.5.175.77') +client_ipv4_bad = cluster.add_instance('client_ipv4_bad', main_configs=[], user_configs=[], ipv4_address='10.5.173.10') -node_ipv6 = cluster.add_instance('node_ipv6', config_dir="configs", main_configs=["configs/config_ipv6.xml"], user_configs=['configs/users_ipv6.xml'], ipv6_address='2001:3984:3989::1:1000') -client_ipv6_ok = cluster.add_instance('client_ipv6_ok', config_dir="configs", ipv6_address='2001:3984:3989::5555') -client_ipv6_ok_direct = cluster.add_instance('client_ipv6_ok_direct', config_dir="configs", ipv6_address='2001:3984:3989::1:1111') -client_ipv6_bad = cluster.add_instance('client_ipv6_bad', config_dir="configs", ipv6_address='2001:3984:3989::1:1112') +node_ipv6 = cluster.add_instance('node_ipv6', main_configs=["configs/config_ipv6.xml"], user_configs=['configs/users_ipv6.xml'], ipv6_address='2001:3984:3989::1:1000') +client_ipv6_ok = cluster.add_instance('client_ipv6_ok', main_configs=[], user_configs=[], ipv6_address='2001:3984:3989::5555') +client_ipv6_ok_direct = cluster.add_instance('client_ipv6_ok_direct', main_configs=[], user_configs=[], ipv6_address='2001:3984:3989::1:1111') +client_ipv6_bad = cluster.add_instance('client_ipv6_bad', main_configs=[], user_configs=[], ipv6_address='2001:3984:3989::1:1112') @pytest.fixture(scope="module") diff --git a/tests/integration/test_user_zero_database_access/test_user_zero_database_access.py b/tests/integration/test_user_zero_database_access/test_user_zero_database_access.py index f3d57e2e174..3af5c18544a 100644 --- a/tests/integration/test_user_zero_database_access/test_user_zero_database_access.py +++ b/tests/integration/test_user_zero_database_access/test_user_zero_database_access.py @@ -5,7 +5,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', config_dir="configs") +node = cluster.add_instance('node', user_configs=["configs/users.xml"]) @pytest.fixture(scope="module") diff --git a/tests/integration/test_zookeeper_config/configs_secure/conf.d/ssl_conf.xml b/tests/integration/test_zookeeper_config/configs_secure/conf.d/ssl_conf.xml index 5e6f5f37624..50303fb70cc 100644 --- a/tests/integration/test_zookeeper_config/configs_secure/conf.d/ssl_conf.xml +++ b/tests/integration/test_zookeeper_config/configs_secure/conf.d/ssl_conf.xml @@ -1,8 +1,8 @@ - /etc/clickhouse-server/client.crt - /etc/clickhouse-server/client.key + /etc/clickhouse-server/config.d/client.crt + /etc/clickhouse-server/config.d/client.key true true sslv2,sslv3 diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 5ee6a8af021..086b9ac0c73 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -12,8 +12,8 @@ def test_chroot_with_same_root(): cluster_1 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml') cluster_2 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml') - node1 = cluster_1.add_instance('node1', config_dir='configs', with_zookeeper=True, zookeeper_use_tmpfs=False) - node2 = cluster_2.add_instance('node2', config_dir='configs', with_zookeeper=True, zookeeper_use_tmpfs=False) + node1 = cluster_1.add_instance('node1', main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"], with_zookeeper=True, zookeeper_use_tmpfs=False) + node2 = cluster_2.add_instance('node2', main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"], with_zookeeper=True, zookeeper_use_tmpfs=False) nodes = [node1, node2] def create_zk_root(zk): @@ -51,8 +51,8 @@ def test_chroot_with_different_root(): cluster_1 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml') cluster_2 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_b.xml') - node1 = cluster_1.add_instance('node1', config_dir='configs', with_zookeeper=True, zookeeper_use_tmpfs=False) - node2 = cluster_2.add_instance('node2', config_dir='configs', with_zookeeper=True, zookeeper_use_tmpfs=False) + node1 = cluster_1.add_instance('node1', main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"], with_zookeeper=True, zookeeper_use_tmpfs=False) + node2 = cluster_2.add_instance('node2', main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_b.xml"], with_zookeeper=True, zookeeper_use_tmpfs=False) nodes = [node1, node2] def create_zk_roots(zk): @@ -90,8 +90,8 @@ def test_identity(): cluster_1 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_with_password.xml') cluster_2 = ClickHouseCluster(__file__) - node1 = cluster_1.add_instance('node1', config_dir='configs', with_zookeeper=True, zookeeper_use_tmpfs=False) - node2 = cluster_2.add_instance('node2', config_dir='configs', with_zookeeper=True, zookeeper_use_tmpfs=False) + node1 = cluster_1.add_instance('node1', main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_with_password.xml"], with_zookeeper=True, zookeeper_use_tmpfs=False) + node2 = cluster_2.add_instance('node2', main_configs=["configs/remote_servers.xml"], with_zookeeper=True, zookeeper_use_tmpfs=False) try: cluster_1.start() @@ -145,10 +145,12 @@ def test_secure_connection(): ) docker_compose.close() - node1 = cluster.add_instance('node1', config_dir='configs_secure', with_zookeeper=True, - zookeeper_docker_compose_path=docker_compose.name, zookeeper_use_tmpfs=False) - node2 = cluster.add_instance('node2', config_dir='configs_secure', with_zookeeper=True, - zookeeper_docker_compose_path=docker_compose.name, zookeeper_use_tmpfs=False) + node1 = cluster.add_instance('node1', main_configs=["configs_secure/client.crt", "configs_secure/client.key", + "configs_secure/conf.d/remote_servers.xml", "configs_secure/conf.d/ssl_conf.xml"], + with_zookeeper=True, zookeeper_docker_compose_path=docker_compose.name, zookeeper_use_tmpfs=False) + node2 = cluster.add_instance('node2', main_configs=["configs_secure/client.crt", "configs_secure/client.key", + "configs_secure/conf.d/remote_servers.xml", "configs_secure/conf.d/ssl_conf.xml"], + with_zookeeper=True, zookeeper_docker_compose_path=docker_compose.name, zookeeper_use_tmpfs=False) try: cluster.start() From ad740fc7daddafd90d59a87a48fe80da904f25b8 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Aug 2020 12:26:01 +0300 Subject: [PATCH 0047/1911] copy dictionaries --- tests/integration/helpers/dictonaries | 1 - .../dictonaries/decimals_dictionary.xml | 197 +++++++ .../helpers/dictonaries/ints_dictionary.xml | 514 ++++++++++++++++++ .../dictonaries/strings_dictionary.xml | 209 +++++++ 4 files changed, 920 insertions(+), 1 deletion(-) delete mode 120000 tests/integration/helpers/dictonaries create mode 100644 tests/integration/helpers/dictonaries/decimals_dictionary.xml create mode 100644 tests/integration/helpers/dictonaries/ints_dictionary.xml create mode 100644 tests/integration/helpers/dictonaries/strings_dictionary.xml diff --git a/tests/integration/helpers/dictonaries b/tests/integration/helpers/dictonaries deleted file mode 120000 index b33ab3b1e87..00000000000 --- a/tests/integration/helpers/dictonaries +++ /dev/null @@ -1 +0,0 @@ -../../config/dict_examples/ \ No newline at end of file diff --git a/tests/integration/helpers/dictonaries/decimals_dictionary.xml b/tests/integration/helpers/dictonaries/decimals_dictionary.xml new file mode 100644 index 00000000000..f728fa774a7 --- /dev/null +++ b/tests/integration/helpers/dictonaries/decimals_dictionary.xml @@ -0,0 +1,197 @@ + + + flat_decimals + + + localhost + 9000 + default + + system + decimals
+
+ + 0 + + + + + + key + + + d32 + Decimal32(4) + 0 + + + d64 + Decimal64(6) + 0 + + + d128 + Decimal128(1) + 0 + + +
+ + + hashed_decimals + + + localhost + 9000 + default + + system + decimals
+
+ + 0 + + + + + + key + + + d32 + Decimal32(4) + 0 + + + d64 + Decimal64(6) + 0 + + + d128 + Decimal128(1) + 0 + + +
+ + + cache_decimals + + + localhost + 9000 + default + + system + decimals
+
+ + 0 + + 1000 + + + + key + + + d32 + Decimal32(4) + 0 + + + d64 + Decimal64(6) + 0 + + + d128 + Decimal128(1) + 0 + + +
+ + + complex_hashed_decimals + + + localhost + 9000 + default + + system + decimals
+
+ + 0 + + + + + + + key + UInt64 + + + + d32 + Decimal32(4) + 0 + + + d64 + Decimal64(6) + 0 + + + d128 + Decimal128(1) + 0 + + +
+ + + complex_cache_decimals + + + localhost + 9000 + default + + system + decimals
+
+ + 0 + + 1000 + + + + + key + UInt64 + + + + d32 + Decimal32(4) + 0 + + + d64 + Decimal64(6) + 0 + + + d128 + Decimal128(1) + 0 + + +
+
diff --git a/tests/integration/helpers/dictonaries/ints_dictionary.xml b/tests/integration/helpers/dictonaries/ints_dictionary.xml new file mode 100644 index 00000000000..a22dab8933c --- /dev/null +++ b/tests/integration/helpers/dictonaries/ints_dictionary.xml @@ -0,0 +1,514 @@ + + + flat_ints + + + localhost + 9000 + default + + system + ints
+
+ + 0 + + + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + hashed_ints + + + localhost + 9000 + default + + system + ints
+
+ + 0 + + + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + hashed_sparse_ints + + + localhost + 9000 + default + + system + ints
+
+ + 0 + + + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + cache_ints + + + localhost + 9000 + default + + system + ints
+
+ + 0 + + 1000 + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + complex_hashed_ints + + + localhost + 9000 + default + + system + ints
+
+ + 0 + + + + + + + key + UInt64 + + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + complex_cache_ints + + + localhost + 9000 + default + + system + ints
+
+ + 0 + + 1000 + + + + + key + UInt64 + + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + +one_cell_cache_ints + + + localhost + 9000 + default + + test_01054 + ints
+
+ +0 + + 1 + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + + one_cell_cache_ints_overflow + + + localhost + 9000 + default + + test_01054_overflow + ints
+
+ + 0 + + 1 + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ +
\ No newline at end of file diff --git a/tests/integration/helpers/dictonaries/strings_dictionary.xml b/tests/integration/helpers/dictonaries/strings_dictionary.xml new file mode 100644 index 00000000000..c5643eecb68 --- /dev/null +++ b/tests/integration/helpers/dictonaries/strings_dictionary.xml @@ -0,0 +1,209 @@ + + + flat_strings + + + localhost + 9000 + default + + system + strings
+
+ + 0 + + + + + + key + + + str + String + + + +
+ + + hashed_strings + + + localhost + 9000 + default + + system + strings
+
+ + 0 + + + + + + key + + + str + String + + + +
+ + + cache_strings + + + localhost + 9000 + default + + system + strings
+
+ + 0 + + 1000 + + + + key + + + str + String + + + +
+ + + complex_hashed_strings + + + localhost + 9000 + default + + system + strings
+
+ + 0 + + + + + + + key + UInt64 + + + + str + String + + + +
+ + + complex_cache_strings + + + localhost + 9000 + default + + system + strings
+
+ + 0 + + 1000 + + + + + key + UInt64 + + + + str + String + + + +
+ + + complex_hashed_strings_key + + + localhost + 9000 + default + + system + strings
+
+ + 0 + + + + + + + str + String + + + + key + UInt64 + 0 + + +
+ + + complex_cache_strings_key + + + localhost + 9000 + default + + system + strings
+
+ + 0 + + 1000 + + + + + str + String + + + + key + UInt64 + 0 + + +
+
From 730056a9f0167c1b41f57493acc7c1914eb76c77 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Aug 2020 15:22:31 +0300 Subject: [PATCH 0048/1911] fix --- docker/test/integration/runner/dockerd-entrypoint.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 9abf3bde53d..c38260279ed 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -19,8 +19,7 @@ set -e echo "Start tests" export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/clickhouse export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse -export CLICKHOUSE_TESTS_CONFIG_DIR=/clickhouse-config -export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-base-config +export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-config export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} From 73e9f4d4210449d943dcd461bacf4524b211d0cd Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 13 Aug 2020 21:20:47 +0300 Subject: [PATCH 0049/1911] update test_materialize_mysql_database test --- .../runner/compose/docker_compose_mysql.yml | 1 + .../compose/docker_compose_mysql_8_0.yml | 0 .../composes/mysql_5_7_compose.yml | 10 ------- .../materialize_with_ddl.py | 6 ++-- .../test_materialize_mysql_database/test.py | 30 +++++++++---------- 5 files changed, 19 insertions(+), 28 deletions(-) rename tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml => docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml (100%) delete mode 100644 tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml diff --git a/docker/test/integration/runner/compose/docker_compose_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql.yml index 2e3afce117d..cef781f95c4 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql.yml @@ -7,3 +7,4 @@ services: MYSQL_ROOT_PASSWORD: clickhouse ports: - 3308:3306 + command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' diff --git a/tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml b/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml similarity index 100% rename from tests/integration/test_materialize_mysql_database/composes/mysql_8_0_compose.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml diff --git a/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml b/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml deleted file mode 100644 index bfc5b6a9538..00000000000 --- a/tests/integration/test_materialize_mysql_database/composes/mysql_5_7_compose.yml +++ /dev/null @@ -1,10 +0,0 @@ -version: '2.3' -services: - mysql5_7: - image: mysql:5.7 - restart: always - environment: - MYSQL_ROOT_PASSWORD: clickhouse - ports: - - 33307:3306 - command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index cc3a8f82fe1..26f8e9416ba 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -177,7 +177,7 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_1 INT NOT NULL") mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ("0" if service_name == "mysql5_7" else "(id)")) + mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ("0" if service_name == "mysql1" else "(id)")) # create mapping clickhouse_node.query( @@ -193,9 +193,9 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node mysql_node.query("ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query( "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( - "0" if service_name == "mysql5_7" else "(id)")) + "0" if service_name == "mysql1" else "(id)")) - default_expression = "DEFAULT\t0" if service_name == "mysql5_7" else "DEFAULT\tid" + default_expression = "DEFAULT\t0" if service_name == "mysql1" else "DEFAULT\tid" check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t" + default_expression + "\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 08baf87e69f..dceacc1d1e2 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -6,12 +6,12 @@ import pymysql.cursors import pytest import materialize_with_ddl -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, get_docker_compose_path -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DOCKER_COMPOSE_PATH = get_docker_compose_path() cluster = ClickHouseCluster(__file__) -clickhouse_node = cluster.add_instance('node1', config_dir="configs", with_mysql=False) +clickhouse_node = cluster.add_instance('node1', user_configs=["configs/users.xml"], with_mysql=False) @pytest.fixture(scope="module") @@ -61,8 +61,8 @@ class MySQLNodeInstance: @pytest.fixture(scope="module") def started_mysql_5_7(): - mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33307) - docker_compose = os.path.join(SCRIPT_DIR, 'composes', 'mysql_5_7_compose.yml') + mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 3308) + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql.yml') try: subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d']) @@ -76,7 +76,7 @@ def started_mysql_5_7(): @pytest.fixture(scope="module") def started_mysql_8_0(): mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33308) - docker_compose = os.path.join(SCRIPT_DIR, 'composes', 'mysql_8_0_compose.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_8_0.yml') try: subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d']) @@ -88,7 +88,7 @@ def started_mysql_8_0(): def test_materialize_database_dml_with_mysql_5_7(started_cluster, started_mysql_5_7): - materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") + materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") def test_materialize_database_dml_with_mysql_8_0(started_cluster, started_mysql_8_0): @@ -96,15 +96,15 @@ def test_materialize_database_dml_with_mysql_8_0(started_cluster, started_mysql_ def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_5_7): - materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") - materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") - materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") - materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") - materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") + materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") # mysql 5.7 cannot support alter rename column - # materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") - materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") - materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") + # materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_8_0): From a79c3175a7bda80ceb541ed60c8580de08a7445b Mon Sep 17 00:00:00 2001 From: it1804 Date: Fri, 14 Aug 2020 02:00:12 +0500 Subject: [PATCH 0050/1911] Allow authenticate Redis with requirepass option --- src/Dictionaries/RedisDictionarySource.cpp | 20 +++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 8794f0620e2..030ee2b1a06 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -51,12 +51,14 @@ namespace DB const String & host_, UInt16 port_, UInt8 db_index_, + const String & password_, RedisStorageType storage_type_, const Block & sample_block_) : dict_struct{dict_struct_} , host{host_} , port{port_} , db_index{db_index_} + , password{password_} , storage_type{storage_type_} , sample_block{sample_block_} , client{std::make_shared(host, port)} @@ -77,16 +79,22 @@ namespace DB ErrorCodes::INVALID_CONFIG_PARAMETER}; // suppose key[0] is primary key, key[1] is secondary key } + if (!password.empty()) + { + RedisCommand command("AUTH"); + command << password; + String reply = client->execute(command); + if (reply != "OK") + throw Exception{"Authentication failed with reason " + + reply, ErrorCodes::INTERNAL_REDIS_ERROR}; + } if (db_index != 0) { RedisCommand command("SELECT"); - // Use poco's Int64, because it is defined as long long, and on - // MacOS, for the purposes of template instantiation, this type is - // distinct from int64_t, which is our Int64. - command << static_cast(db_index); + command << std::to_string(db_index); String reply = client->execute(command); - if (reply != "+OK\r\n") + if (reply != "OK") throw Exception{"Selecting database with index " + DB::toString(db_index) + " failed with reason " + reply, ErrorCodes::INTERNAL_REDIS_ERROR}; } @@ -103,6 +111,7 @@ namespace DB config_.getString(config_prefix_ + ".host"), config_.getUInt(config_prefix_ + ".port"), config_.getUInt(config_prefix_ + ".db_index", 0), + config_.getString(config_prefix_ + ".password",""), parseStorageType(config_.getString(config_prefix_ + ".storage_type", "")), sample_block_) { @@ -114,6 +123,7 @@ namespace DB other.host, other.port, other.db_index, + other.password, other.storage_type, other.sample_block} { From a1c0c52c5bdda2358139d712352c706f4dd20086 Mon Sep 17 00:00:00 2001 From: it1804 Date: Fri, 14 Aug 2020 02:01:25 +0500 Subject: [PATCH 0051/1911] Allow authenticate Redis with requirepass option --- src/Dictionaries/RedisDictionarySource.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Dictionaries/RedisDictionarySource.h b/src/Dictionaries/RedisDictionarySource.h index b30c428cb2d..75dcc2fb081 100644 --- a/src/Dictionaries/RedisDictionarySource.h +++ b/src/Dictionaries/RedisDictionarySource.h @@ -41,6 +41,7 @@ namespace ErrorCodes const std::string & host, UInt16 port, UInt8 db_index, + const std::string & password, RedisStorageType storage_type, const Block & sample_block); @@ -91,6 +92,7 @@ namespace ErrorCodes const std::string host; const UInt16 port; const UInt8 db_index; + const std::string password; const RedisStorageType storage_type; Block sample_block; From fac881a6f0f50005fffa95e4ef77c071bb2c5d0e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 14 Aug 2020 18:51:28 +0300 Subject: [PATCH 0052/1911] finally remove config_dir --- tests/integration/helpers/cluster.py | 21 +++++-------------- .../test_distributed_ddl/cluster.py | 16 +++++--------- .../test_polymorphic_parts/test.py | 8 +++---- .../test.py | 2 -- tests/integration/test_system_merges/test.py | 2 -- 5 files changed, 14 insertions(+), 35 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 69db0c0fb10..9ce84478c7a 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -89,7 +89,7 @@ class ClickHouseCluster: these directories will contain logs, database files, docker-compose config, ClickHouse configs etc. """ - def __init__(self, base_path, name=None, base_config_dir=None, config_dir=None, server_bin_path=None, client_bin_path=None, + def __init__(self, base_path, name=None, base_config_dir=None, server_bin_path=None, client_bin_path=None, odbc_bridge_bin_path=None, zookeeper_config_path=None, custom_dockerd_host=None): for param in os.environ.keys(): print "ENV %40s %s" % (param,os.environ[param]) @@ -98,8 +98,6 @@ class ClickHouseCluster: self.base_config_dir = base_config_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR', '/etc/clickhouse-server/') - self.config_dir = config_dir or os.environ.get('CLICKHOUSE_TESTS_CONFIG_DIR', - '/etc/clickhouse-server/') self.server_bin_path = p.realpath( server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse')) self.odbc_bridge_bin_path = p.realpath(odbc_bridge_bin_path or get_odbc_bridge_path()) @@ -159,7 +157,7 @@ class ClickHouseCluster: self.docker_client = None self.is_up = False - print "CLUSTER INIT base_config_dir:{} config_dir:{}".format(self.base_config_dir, self.config_dir) + print "CLUSTER INIT base_config_dir:{}".format(self.base_config_dir) def get_client_cmd(self): cmd = self.client_bin_path @@ -167,7 +165,7 @@ class ClickHouseCluster: cmd += " client" return cmd - def add_instance(self, name, base_config_dir=None, config_dir=None, main_configs=None, user_configs=None, dictionaries = None, macros=None, + 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_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, @@ -177,7 +175,6 @@ class ClickHouseCluster: """Add an instance to the cluster. name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. - config_dir - a directory with config files which content will be copied to /etc/clickhouse-server/ directory base_config_dir - a directory with config.xml and users.xml files which will be copied to /etc/clickhouse-server/ directory main_configs - a list of config files that will be added to config.d/ directory user_configs - a list of config files that will be added to users.d/ directory @@ -192,8 +189,7 @@ class ClickHouseCluster: instance = ClickHouseInstance( self, self.base_dir, name, base_config_dir if base_config_dir else self.base_config_dir, - config_dir if config_dir else self.config_dir, main_configs or [], user_configs or [], dictionaries or [], - macros or {}, with_zookeeper, + main_configs or [], user_configs or [], dictionaries or [], macros or {}, with_zookeeper, self.zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra, self.server_bin_path, self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname, env_variables=env_variables or {}, image=image, stay_alive=stay_alive, ipv4_address=ipv4_address, @@ -765,7 +761,7 @@ services: class ClickHouseInstance: def __init__( - self, cluster, base_path, name, base_config_dir, config_dir, custom_main_configs, custom_user_configs, custom_dictionaries, + 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_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, @@ -780,7 +776,6 @@ class ClickHouseInstance: self.tmpfs = tmpfs or [] self.base_config_dir = p.abspath(p.join(base_path, base_config_dir)) if base_config_dir else None - self.config_dir = p.abspath(p.join(base_path, config_dir)) if config_dir else None self.custom_main_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_main_configs] 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] @@ -1126,10 +1121,6 @@ class ClickHouseInstance: if self.with_zookeeper: shutil.copy(self.zookeeper_config_path, conf_d_dir) - # print "Copy config dir {} to {}".format(self.config_dir, instance_config_dir) - # if self.config_dir: - # distutils.dir_util.copy_tree(self.config_dir, instance_config_dir) - # 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: @@ -1139,8 +1130,6 @@ class ClickHouseInstance: for path in self.custom_user_config_paths: shutil.copy(path, users_d_dir) - - self.config_dir # Copy dictionaries configs to configs/dictionaries for path in self.custom_dictionaries_paths: shutil.copy(path, dictionaries_dir) diff --git a/tests/integration/test_distributed_ddl/cluster.py b/tests/integration/test_distributed_ddl/cluster.py index d7cb3d81c82..b3a0513b799 100644 --- a/tests/integration/test_distributed_ddl/cluster.py +++ b/tests/integration/test_distributed_ddl/cluster.py @@ -17,18 +17,12 @@ class ClickHouseClusterWithDDLHelpers(ClickHouseCluster): def prepare(self, replace_hostnames_with_ips=True): try: - main_configs = [os.path.join(self.test_config_dir, "config.d/clusters.xml"), - os.path.join(self.test_config_dir, "config.d/zookeeper_session_timeout.xml"), - os.path.join(self.test_config_dir, "config.d/macro.xml"), - os.path.join(self.test_config_dir, "config.d/query_log.xml"), - os.path.join(self.test_config_dir, "config.d/ddl.xml")] - user_configs = [os.path.join(self.test_config_dir, "users.d/restricted_user.xml"), - os.path.join(self.test_config_dir, "users.d/query_log.xml")] + main_configs_files = ["clusters.xml", "zookeeper_session_timeout.xml", "macro.xml"), + "query_log.xml","ddl.xml"] + main_configs = [os.path.join(self.test_config_dir, "config.d", f) for f in main_configs_files)] + user_configs = [os.path.join(self.test_config_dir, "users.d", f) for f in ["restricted_user.xml", "query_log.xml"]] if self.test_config_dir == "configs_secure": - main_configs += [os.path.join(self.test_config_dir, "server.crt"), - os.path.join(self.test_config_dir, "server.key"), - os.path.join(self.test_config_dir, "dhparam.pem"), - os.path.join(self.test_config_dir, "config.d/ssl_conf.xml")] + main_configs += [os.path.join(self.test_config_dir, f) for i in ["server.crt", "server.key", "dhparam.pem", "config.d/ssl_conf.xml"]] for i in xrange(4): self.add_instance( 'ch{}'.format(i+1), diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index 7fd29216680..e6c093ad414 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -71,11 +71,11 @@ node6 = cluster.add_instance('node6', main_configs=['configs/compact_parts.xml'] settings_in_memory = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 256} -node9 = cluster.add_instance('node9', config_dir="configs", with_zookeeper=True, stay_alive=True) -node10 = cluster.add_instance('node10', config_dir="configs", with_zookeeper=True) +node9 = cluster.add_instance('node9', with_zookeeper=True, stay_alive=True) +node10 = cluster.add_instance('node10', with_zookeeper=True) -node11 = cluster.add_instance('node11', config_dir="configs", main_configs=['configs/do_not_merge.xml'], with_zookeeper=True, stay_alive=True) -node12 = cluster.add_instance('node12', config_dir="configs", main_configs=['configs/do_not_merge.xml'], with_zookeeper=True, stay_alive=True) +node11 = cluster.add_instance('node11', main_configs=['configs/do_not_merge.xml'], with_zookeeper=True, stay_alive=True) +node12 = cluster.add_instance('node12', main_configs=['configs/do_not_merge.xml'], with_zookeeper=True, stay_alive=True) @pytest.fixture(scope="module") def start_cluster(): diff --git a/tests/integration/test_reloading_storage_configuration/test.py b/tests/integration/test_reloading_storage_configuration/test.py index c9effcdd67a..a30d4029d7c 100644 --- a/tests/integration/test_reloading_storage_configuration/test.py +++ b/tests/integration/test_reloading_storage_configuration/test.py @@ -14,7 +14,6 @@ import helpers.cluster cluster = helpers.cluster.ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', - config_dir='configs', main_configs=['configs/logs_config.xml'], with_zookeeper=True, stay_alive=True, @@ -22,7 +21,6 @@ node1 = cluster.add_instance('node1', macros={"shard": 0, "replica": 1} ) node2 = cluster.add_instance('node2', - config_dir='configs', main_configs=['configs/logs_config.xml'], with_zookeeper=True, stay_alive=True, diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index 8e3714bc23b..15e5b1c0835 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -6,13 +6,11 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', - config_dir='configs', main_configs=['configs/logs_config.xml'], with_zookeeper=True, macros={"shard": 0, "replica": 1} ) node2 = cluster.add_instance('node2', - config_dir='configs', main_configs=['configs/logs_config.xml'], with_zookeeper=True, macros={"shard": 0, "replica": 2} ) From c6fdeb6c021b0d9724608925513c3ef657e5a232 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 15 Aug 2020 06:50:53 +0000 Subject: [PATCH 0053/1911] Better --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 8 +- src/Storages/RabbitMQ/RabbitMQHandler.cpp | 6 +- .../ReadBufferFromRabbitMQConsumer.cpp | 122 +++--- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 51 ++- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 88 ++-- src/Storages/RabbitMQ/StorageRabbitMQ.h | 12 + .../WriteBufferToRabbitMQProducer.cpp | 104 +++-- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 10 +- .../integration/test_storage_rabbitmq/test.py | 396 ++++++++---------- 9 files changed, 419 insertions(+), 378 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index 589f5b39d2e..e26645a1168 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -22,7 +22,7 @@ RabbitMQBlockInputStream::RabbitMQBlockInputStream( , column_names(columns) , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) , virtual_header(metadata_snapshot->getSampleBlockForColumns( - {"_exchange_name", "_consumer_tag", "_delivery_tag", "_redelivered"}, storage.getVirtuals(), storage.getStorageID())) + {"_exchange_name", "_channel_id", "_delivery_tag", "_redelivered"}, storage.getVirtuals(), storage.getStorageID())) { } @@ -128,16 +128,16 @@ Block RabbitMQBlockInputStream::readImpl() if (new_rows) { auto exchange_name = storage.getExchange(); - auto consumer_tag = buffer->getConsumerTag(); + auto channel_id = buffer->getChannelID(); auto delivery_tag = buffer->getDeliveryTag(); auto redelivered = buffer->getRedelivered(); - buffer->updateNextDeliveryTag(delivery_tag); + buffer->updateAckTracker({delivery_tag, channel_id}); for (size_t i = 0; i < new_rows; ++i) { virtual_columns[0]->insert(exchange_name); - virtual_columns[1]->insert(consumer_tag); + virtual_columns[1]->insert(channel_id); virtual_columns[2]->insert(delivery_tag); virtual_columns[3]->insert(redelivered); } diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.cpp b/src/Storages/RabbitMQ/RabbitMQHandler.cpp index d6b6ab440b2..835ded1718c 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.cpp +++ b/src/Storages/RabbitMQ/RabbitMQHandler.cpp @@ -20,18 +20,16 @@ RabbitMQHandler::RabbitMQHandler(uv_loop_t * loop_, Poco::Logger * log_) : ///Method that is called when the connection ends up in an error state. void RabbitMQHandler::onError(AMQP::TcpConnection * connection, const char * message) { - connection_running.store(false); LOG_ERROR(log, "Library error report: {}", message); - + connection_running.store(false); if (connection) connection->close(); } void RabbitMQHandler::onReady(AMQP::TcpConnection * /* connection */) { - connection_running.store(true); LOG_TRACE(log, "Connection is ready"); - + connection_running.store(true); loop_state.store(Loop::RUN); } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index d12d08fad25..833382f354b 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -21,7 +21,8 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ChannelPtr setup_channel_, HandlerPtr event_handler_, const String & exchange_name_, - size_t channel_id_, + size_t channel_id_base_, + const String & channel_base_, const String & queue_base_, Poco::Logger * log_, char row_delimiter_, @@ -34,14 +35,15 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( , setup_channel(setup_channel_) , event_handler(event_handler_) , exchange_name(exchange_name_) - , channel_id(channel_id_) + , channel_base(channel_base_) + , channel_id_base(channel_id_base_) , queue_base(queue_base_) , hash_exchange(hash_exchange_) , num_queues(num_queues_) + , deadletter_exchange(deadletter_exchange_) , log(log_) , row_delimiter(row_delimiter_) , stopped(stopped_) - , deadletter_exchange(deadletter_exchange_) , received(QUEUE_SIZE * num_queues) { for (size_t queue_id = 0; queue_id < num_queues; ++queue_id) @@ -49,27 +51,32 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( consumer_channel->onReady([&]() { + channel_id = channel_base + "_" + std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++); + LOG_TRACE(log, "Channel {} is created", channel_id); + consumer_channel->onError([&](const char * message) { - LOG_ERROR(log, "Consumer {} error: {}", channel_id, message); + LOG_ERROR(log, "Channel {} error: {}", channel_id, message); channel_error.store(true); }); + updateAckTracker(AckTracker()); subscribe(); + + channel_error.store(false); }); } ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer() { - consumer_channel->close(); BufferBase::set(nullptr, 0, 0); } void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) { - std::atomic bindings_created = false, bindings_error = false; + std::atomic binding_created = false; auto success_callback = [&](const std::string & queue_name, int msgcount, int /* consumercount */) { @@ -83,22 +90,20 @@ void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) * done between client's exchange and local bridge exchange. Binding key must be a string integer in case of hash exchange, for * fanout exchange it can be arbitrary. */ - setup_channel->bindQueue(exchange_name, queue_name, std::to_string(channel_id)) + setup_channel->bindQueue(exchange_name, queue_name, std::to_string(channel_id_base)) .onSuccess([&] { - bindings_created = true; + binding_created = true; }) .onError([&](const char * message) { - bindings_error = true; - LOG_ERROR(log, "Failed to create queue binding. Reason: {}", message); + throw Exception("Failed to create queue binding. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); }; auto error_callback([&](const char * message) { - bindings_error = true; - LOG_ERROR(log, "Failed to declare queue on the channel. Reason: {}", message); + throw Exception("Failed to declare queue. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); AMQP::Table queue_settings; @@ -110,10 +115,10 @@ void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id) /* The first option not just simplifies queue_name, but also implements the possibility to be able to resume reading from one * specific queue when its name is specified in queue_base setting. */ - const String queue_name = !hash_exchange ? queue_base : queue_base + "_" + std::to_string(channel_id) + "_" + std::to_string(queue_id); + const String queue_name = !hash_exchange ? queue_base : queue_base + "_" + std::to_string(channel_id_base) + "_" + std::to_string(queue_id); setup_channel->declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); - while (!bindings_created && !bindings_error) + while (!binding_created) { iterateEventLoop(); } @@ -125,11 +130,9 @@ void ReadBufferFromRabbitMQConsumer::subscribe() for (const auto & queue_name : queues) { consumer_channel->consume(queue_name) - .onSuccess([&](const std::string & consumer) + .onSuccess([&](const std::string & /* consumer_tag */) { - if (consumer_tag.empty()) - consumer_tag = consumer; - LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name); + LOG_TRACE(log, "Consumer on channel {} is subscribed to queue {}", channel_id, queue_name); }) .onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered) { @@ -139,12 +142,12 @@ void ReadBufferFromRabbitMQConsumer::subscribe() if (row_delimiter != '\0') message_received += row_delimiter; - received.push({delivery_tag, message_received, redelivered}); + received.push({message_received, redelivered, AckTracker(delivery_tag, channel_id)}); } }) .onError([&](const char * message) { - LOG_ERROR(log, "Consumer {} failed. Reason: {}", channel_id, message); + LOG_ERROR(log, "Consumer failed on channel {}. Reason: {}", channel_id, message); }); } } @@ -152,16 +155,63 @@ void ReadBufferFromRabbitMQConsumer::subscribe() void ReadBufferFromRabbitMQConsumer::ackMessages() { - UInt64 delivery_tag = last_inserted_delivery_tag; - if (delivery_tag && delivery_tag > prev_tag) + /* Delivery tags are scoped per channel, so if channel fails, then all previous delivery tags become invalid. Also this check ensures + * that there is no data race with onReady callback in restoreChannel() (they can be called at the same time from different threads). + * And there is no need to synchronize this method with updateAckTracker() as they are not supposed to be called at the same time. + */ + if (channel_error.load()) + return; + + AckTracker record = last_inserted_record; + + /// Do not send ack to server if message's channel is not the same as current running channel. + if (record.channel_id == channel_id && record.delivery_tag && record.delivery_tag > prev_tag) { - prev_tag = delivery_tag; - consumer_channel->ack(prev_tag, AMQP::multiple); /// Will ack all up to last tag staring from last acked. - LOG_TRACE(log, "Consumer {} acknowledged messages with deliveryTags up to {}", channel_id, prev_tag); + consumer_channel->ack(record.delivery_tag, AMQP::multiple); /// Will ack all up to last tag starting from last acked. + prev_tag = record.delivery_tag; + + LOG_TRACE(log, "Consumer acknowledged messages with deliveryTags up to {} on the channel {}", record.delivery_tag, channel_id); } } +void ReadBufferFromRabbitMQConsumer::updateAckTracker(AckTracker record) +{ + /* This method can be called from readImpl and from channel->onError() callback, but channel_error check ensures that it is not done + * at the same time, so no synchronization needed. + */ + if (record.delivery_tag && channel_error.load()) + return; + + if (!record.delivery_tag) + prev_tag = 0; + + last_inserted_record = record; +} + + +void ReadBufferFromRabbitMQConsumer::restoreChannel(ChannelPtr new_channel) +{ + consumer_channel = std::move(new_channel); + consumer_channel->onReady([&]() + { + channel_id = channel_base + "_" + std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++); + LOG_TRACE(log, "Channel {} is created", channel_id); + + consumer_channel->onError([&](const char * message) + { + LOG_ERROR(log, "Channel {} error: {}", channel_id, message); + channel_error.store(true); + }); + + updateAckTracker(AckTracker()); + subscribe(); + + channel_error.store(false); + }); +} + + void ReadBufferFromRabbitMQConsumer::iterateEventLoop() { event_handler->iterateLoop(); @@ -185,26 +235,4 @@ bool ReadBufferFromRabbitMQConsumer::nextImpl() return false; } - -void ReadBufferFromRabbitMQConsumer::restoreChannel(ChannelPtr new_channel) -{ - if (consumer_channel->usable()) - return; - - consumer_channel = std::move(new_channel); - consumer_channel->onReady([&]() - { - LOG_TRACE(log, "Channel {} is restored", channel_id); - channel_error.store(false); - consumer_channel->onError([&](const char * message) - { - LOG_ERROR(log, "Consumer {} error: {}", consumer_tag, message); - channel_error.store(true); - }); - - subscribe(); - }); -} - - } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index d3f560fad3b..c5643cb59f4 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -27,7 +27,8 @@ public: ChannelPtr setup_channel_, HandlerPtr event_handler_, const String & exchange_name_, - size_t channel_id_, + size_t channel_id_base_, + const String & channel_base_, const String & queue_base_, Poco::Logger * log_, char row_delimiter_, @@ -38,53 +39,65 @@ public: ~ReadBufferFromRabbitMQConsumer() override; - struct MessageData + struct AckTracker { UInt64 delivery_tag; + String channel_id; + + AckTracker() : delivery_tag(0), channel_id("") {} + AckTracker(UInt64 tag, String id) : delivery_tag(tag), channel_id(id) {} + }; + + struct MessageData + { String message; bool redelivered; + AckTracker track; }; void allowNext() { allowed = true; } // Allow to read next message. bool channelUsable() { return !channel_error.load(); } void restoreChannel(ChannelPtr new_channel); - void updateNextDeliveryTag(UInt64 delivery_tag) { last_inserted_delivery_tag = delivery_tag; } - void ackMessages(); - auto getConsumerTag() const { return consumer_tag; } - auto getDeliveryTag() const { return current.delivery_tag; } + void ackMessages(); + void updateAckTracker(AckTracker record); + + auto getChannelID() const { return current.track.channel_id; } + auto getDeliveryTag() const { return current.track.delivery_tag; } auto getRedelivered() const { return current.redelivered; } private: + bool nextImpl() override; + + void bindQueue(size_t queue_id); + void subscribe(); + void iterateEventLoop(); + ChannelPtr consumer_channel; ChannelPtr setup_channel; HandlerPtr event_handler; const String exchange_name; - const size_t channel_id; + const String channel_base; + const size_t channel_id_base; const String queue_base; const bool hash_exchange; const size_t num_queues; + const String deadletter_exchange; Poco::Logger * log; char row_delimiter; bool allowed = true; const std::atomic & stopped; - const String deadletter_exchange; - std::atomic channel_error = false; - - String consumer_tag; - ConcurrentBoundedQueue received; - UInt64 last_inserted_delivery_tag = 0, prev_tag = 0; - MessageData current; + String channel_id; + std::atomic channel_error = true; std::vector queues; + ConcurrentBoundedQueue received; + MessageData current; - bool nextImpl() override; - - void bindQueue(size_t queue_id); - void subscribe(); - void iterateEventLoop(); + AckTracker last_inserted_record; + UInt64 prev_tag = 0, channel_id_counter = 0; }; } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index a477477fd63..f85f7d6b59c 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -110,7 +110,8 @@ StorageRabbitMQ::StorageRabbitMQ( if (!connection->ready()) { - uv_loop_close(loop.get()); + if (!connection->closed()) + connection->close(true); throw Exception("Cannot connect to RabbitMQ", ErrorCodes::CANNOT_CONNECT_RABBITMQ); } @@ -145,7 +146,9 @@ StorageRabbitMQ::StorageRabbitMQ( if (queue_base.empty()) { - /// Make sure that local exchange name is unique for each table and is not the same as client's exchange name + /* Make sure that local exchange name is unique for each table and is not the same as client's exchange name. It also needs to + * be table_name and not just a random string, because local exchanges should be declared the same for same tables. + */ sharding_exchange = exchange_name + "_" + table_name; /* By default without a specified queue name in queue's declaration - its name will be generated by the library, but its better @@ -159,13 +162,19 @@ StorageRabbitMQ::StorageRabbitMQ( { /* In case different tables are used to register multiple consumers to the same queues (so queues are shared between tables) and * at the same time sharding exchange is needed (if there are multiple shared queues), then those tables also need - * to share sharding exchange. + * to share sharding exchange and bridge exchange. */ sharding_exchange = exchange_name + "_" + queue_base; } bridge_exchange = sharding_exchange + "_bridge"; + /* Generate a random string, which will be used for channelID's, which must be unique to tables and to channels within each table. + * (Cannot use table_name here because it must be a different string if table was restored) + */ + unique_strbase = getRandomName(); + + /// One looping task for all consumers as they share the same connection == the same handler == the same event loop looping_task = global_context.getSchedulePool().createTask("RabbitMQLoopingTask", [this]{ loopingFunc(); }); looping_task->deactivate(); @@ -195,21 +204,21 @@ void StorageRabbitMQ::loopingFunc() void StorageRabbitMQ::initExchange() { - /* Declare client's exchange of the specified type and bind it to hash-exchange (if it is not already hash-exchange), which - * will evenly distribute messages between all consumers. + /* Binding scheme is the following: client's exchange -> key bindings by routing key list -> bridge exchange (fanout) -> + * -> sharding exchange (only if needed) -> queues. */ setup_channel->declareExchange(exchange_name, exchange_type, AMQP::durable) .onError([&](const char * message) { throw Exception("Unable to declare exchange. Make sure specified exchange is not already declared. Error: " - + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + + std::string(message), ErrorCodes::LOGICAL_ERROR); }); - /// Bridge exchange is needed to easily disconnect consumer queues. Also simplifies queue bindings a lot. + /// Bridge exchange is needed to easily disconnect consumer queues and also simplifies queue bindings a lot. setup_channel->declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable + AMQP::autodelete) .onError([&](const char * message) { - throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); if (!hash_exchange) @@ -218,20 +227,23 @@ void StorageRabbitMQ::initExchange() return; } - /// Declare exchange for sharding. + /* Change hash property because by default it will be routing key, which has to be an integer, but with support for any exchange + * type - routing keys will not be such. + */ AMQP::Table binding_arguments; binding_arguments["hash-property"] = "message_id"; + /// Declare exchange for sharding. setup_channel->declareExchange(sharding_exchange, AMQP::consistent_hash, AMQP::durable + AMQP::autodelete, binding_arguments) .onError([&](const char * message) { - throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); setup_channel->bindExchange(bridge_exchange, sharding_exchange, routing_keys[0]) .onError([&](const char * message) { - throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); consumer_exchange = sharding_exchange; @@ -260,7 +272,7 @@ void StorageRabbitMQ::bindExchange() }) .onError([&](const char * message) { - throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); } else if (exchange_type == AMQP::ExchangeType::fanout || exchange_type == AMQP::ExchangeType::consistent_hash) @@ -272,7 +284,7 @@ void StorageRabbitMQ::bindExchange() }) .onError([&](const char * message) { - throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); } else @@ -288,7 +300,7 @@ void StorageRabbitMQ::bindExchange() }) .onError([&](const char * message) { - throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ); + throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR); }); } } @@ -302,6 +314,16 @@ void StorageRabbitMQ::bindExchange() void StorageRabbitMQ::unbindExchange() { + /* This is needed because with RabbitMQ can't (without special adjustments) use the same table for reading and writing (alternating them), + * because publishing is done to exchange, publisher never knows to which queues the message will go, every application interested in + * consuming from certain exchange - declares its owns exchange-bound queues, messages go to all such exchange-bound queues, and as + * input streams are always created at startup, then they will also declare its own exchange bound queues, but they will not be visible + * externally - client declares its own exchange-bound queues, from which to consume, so this means that if not disconnecting this local + * queues, then messages will go both ways and in one of them they will remain not consumed. Therefore, if insert query is called, need + * to desconnect local consumers, but then MV cannot be afterwards created on the same table. It can be reverted to allow alternating + * these queries, but it will be ugly and seems pointless because probably nobody uses tables alternating INSERT and MV queries on the + * same table. + */ std::call_once(flag, [&]() { setup_channel->removeExchange(bridge_exchange) @@ -333,8 +355,9 @@ bool StorageRabbitMQ::restoreConnection() /// This lock is to synchronize with getChannel(). std::lock_guard lk(connection_mutex); - if (!connection->usable() || !connection->ready()) + if (!event_handler->connectionRunning()) { + /// Stopping loop now and not right after connection error, because need to run it to let it properly close connection. if (event_handler->getLoopState() == Loop::RUN) { event_handler->updateLoopState(Loop::STOP); @@ -342,12 +365,12 @@ bool StorageRabbitMQ::restoreConnection() heartbeat_task->deactivate(); } - /* connection->close() is called in onError() method (called by the AMQP library when a fatal error occurs on the connection) - * inside event_handler, but it is not closed immediately (firstly, all pending operations are completed, and then an AMQP - * closing-handshake is performed). But cannot open a new connection untill previous one is properly closed). + /* connection->close() is called in onError() method (which is called by the AMQP library when a fatal error occurs on the + * connection) inside event_handler, but it is not closed immediately (firstly, all pending operations are completed, and then + * an AMQP closing-handshake is performed). But cannot open a new connection untill previous one is properly closed. */ size_t cnt_retries = 0; - while (!connection->closed() && ++cnt_retries != (RETRIES_MAX >> 1)) + while (!connection->closed() && ++cnt_retries != RETRIES_MAX) event_handler->iterateLoop(); /// This will force immediate closure if not yet closed. @@ -355,7 +378,8 @@ bool StorageRabbitMQ::restoreConnection() connection->close(true); LOG_TRACE(log, "Trying to restore consumer connection"); - connection = std::make_shared(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); + connection = std::make_shared(event_handler.get(), + AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/")); cnt_retries = 0; while (!connection->ready() && ++cnt_retries != RETRIES_MAX) @@ -386,6 +410,7 @@ ChannelPtr StorageRabbitMQ::getChannel() { std::lock_guard lk(connection_mutex); ChannelPtr new_channel = std::make_shared(connection.get()); + return new_channel; } @@ -466,12 +491,21 @@ void StorageRabbitMQ::shutdown() streaming_task->deactivate(); heartbeat_task->deactivate(); - for (size_t i = 0; i < num_created_consumers; ++i) + connection->close(); + + size_t cnt_retries = 0; + while (!connection->closed() && ++cnt_retries != (RETRIES_MAX >> 1)) { - popReadBuffer(); + event_handler->iterateLoop(); + std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP >> 3)); } - connection->close(); + /// Should actually force closure, if not yet closed, but it generates distracting error logs. + //if (!connection->closed()) + // connection->close(true); + + for (size_t i = 0; i < num_created_consumers; ++i) + popReadBuffer(); } @@ -514,8 +548,8 @@ ConsumerBufferPtr StorageRabbitMQ::createReadBuffer() ChannelPtr consumer_channel = std::make_shared(connection.get()); return std::make_shared( - consumer_channel, setup_channel, event_handler, consumer_exchange, - ++consumer_id, queue_base, log, row_delimiter, hash_exchange, num_queues, + consumer_channel, setup_channel, event_handler, consumer_exchange, ++consumer_id, + unique_strbase, queue_base, log, row_delimiter, hash_exchange, num_queues, deadletter_exchange, stream_cancelled); } @@ -524,7 +558,7 @@ ProducerBufferPtr StorageRabbitMQ::createWriteBuffer() { return std::make_shared( parsed_address, global_context, login_password, routing_keys, exchange_name, exchange_type, - ++producer_id, use_transactional_channel, persistent, log, + ++producer_id, unique_strbase, use_transactional_channel, persistent, log, row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024); } @@ -862,7 +896,7 @@ NamesAndTypesList StorageRabbitMQ::getVirtuals() const { return NamesAndTypesList{ {"_exchange_name", std::make_shared()}, - {"_consumer_tag", std::make_shared()}, + {"_channel_id", std::make_shared()}, {"_delivery_tag", std::make_shared()}, {"_redelivered", std::make_shared()} }; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index cfdbf1c3d94..0960e35d3bf 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -9,8 +9,10 @@ #include #include #include +#include #include #include +#include namespace DB @@ -111,6 +113,7 @@ private: std::mutex mutex; std::vector buffers; /// available buffers for RabbitMQ consumers + String unique_strbase; String sharding_exchange, bridge_exchange, consumer_exchange; std::once_flag flag; size_t producer_id = 0, consumer_id = 0; @@ -135,6 +138,15 @@ private: void pingConnection() { connection->heartbeat(); } bool streamToViews(); bool checkDependencies(const StorageID & table_id); + + String getRandomName() + { + std::uniform_int_distribution distribution('a', 'z'); + String random_str(32, ' '); + for (auto & c : random_str) + c = distribution(thread_local_rng); + return random_str; + } }; } diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 78920bc13c6..2b818f0341f 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -15,7 +15,8 @@ namespace DB static const auto CONNECT_SLEEP = 200; static const auto RETRIES_MAX = 20; -static const auto BATCH = 512; +static const auto BATCH = 10000; +static const auto RETURNED_LIMIT = 50000; WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( std::pair & parsed_address_, @@ -24,8 +25,9 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( const Names & routing_keys_, const String & exchange_name_, const AMQP::ExchangeType exchange_type_, - const size_t channel_id_, - const bool use_tx_, + const size_t channel_id_base_, + const String channel_base_, + const bool use_txn_, const bool persistent_, Poco::Logger * log_, std::optional delimiter, @@ -37,11 +39,12 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( , routing_keys(routing_keys_) , exchange_name(exchange_name_) , exchange_type(exchange_type_) - , channel_id(std::to_string(channel_id_)) - , use_tx(use_tx_) + , channel_id_base(std::to_string(channel_id_base_)) + , channel_base(channel_base_) + , use_txn(use_txn_) , persistent(persistent_) , payloads(BATCH) - , returned(BATCH << 6) + , returned(RETURNED_LIMIT) , log(log_) , delim(delimiter) , max_rows(rows_per_message) @@ -52,14 +55,14 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( uv_loop_init(loop.get()); event_handler = std::make_unique(loop.get(), log); - /* New coonection for each producer buffer because cannot publish from different threads with the same connection. - * (https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/128#issuecomment-300780086) - */ if (setupConnection(false)) setupChannel(); - writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); - writing_task->deactivate(); + if (!use_txn) + { + writing_task = global_context.getSchedulePool().createTask("RabbitMQWritingTask", [this]{ writingFunc(); }); + writing_task->deactivate(); + } if (exchange_type == AMQP::ExchangeType::headers) { @@ -77,6 +80,14 @@ WriteBufferToRabbitMQProducer::~WriteBufferToRabbitMQProducer() { writing_task->deactivate(); connection->close(); + + size_t cnt_retries = 0; + while (!connection->closed() && ++cnt_retries != (RETRIES_MAX >> 1)) + { + event_handler->iterateLoop(); + std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP >> 3)); + } + assert(rows == 0 && chunks.empty()); } @@ -103,7 +114,7 @@ void WriteBufferToRabbitMQProducer::countRow() chunks.clear(); set(nullptr, 0); - if (!use_tx) + if (!use_txn) { /// "publisher confirms" will be used, this is default. ++payload_counter; @@ -125,7 +136,7 @@ bool WriteBufferToRabbitMQProducer::setupConnection(bool reconnecting) { /* connection->close() is called in onError() method (called by the AMQP library when a fatal error occurs on the connection) * inside event_handler, but it is not closed immediately (firstly, all pending operations are completed, and then an AMQP - * closing-handshake is performed). But cannot open a new connection untill previous one is properly closed). + * closing-handshake is performed). But cannot open a new connection untill previous one is properly closed. */ while (!connection->closed() && ++cnt_retries != (RETRIES_MAX >> 1)) event_handler->iterateLoop(); @@ -154,17 +165,19 @@ void WriteBufferToRabbitMQProducer::setupChannel() producer_channel->onError([&](const char * message) { - LOG_ERROR(log, "Producer error: {}", message); + LOG_ERROR(log, "Producer's channel {} error: {}", channel_id, message); /// Channel is not usable anymore. (https://github.com/CopernicaMarketingSoftware/AMQP-CPP/issues/36#issuecomment-125112236) producer_channel->close(); - /// Records that have not received ack/nack from server before channel closure. + /* Save records that have not received ack/nack from server before channel closure. They are removed and pushed back again once + * they are republished because after channel recovery they will acquire new delivery tags, so all previous records become invalid. + */ for (const auto & record : delivery_record) returned.tryPush(record.second); - LOG_DEBUG(log, "Currently {} messages have not been confirmed yet, {} waiting to be published, {} will be republished", - delivery_record.size(), payloads.size(), returned.size()); + LOG_DEBUG(log, "Producer on channel {} hasn't confirmed {} messages, {} waiting to be published", + channel_id, delivery_record.size(), payloads.size()); /// Delivery tags are scoped per channel. delivery_record.clear(); @@ -173,9 +186,10 @@ void WriteBufferToRabbitMQProducer::setupChannel() producer_channel->onReady([&]() { - LOG_DEBUG(log, "Producer channel is ready"); + channel_id = channel_base + "_" + channel_id_base + std::to_string(channel_id_counter++); + LOG_DEBUG(log, "Producer's channel {} is ready", channel_id); - if (use_tx) + if (use_txn) { producer_channel->startTransaction(); } @@ -238,27 +252,31 @@ void WriteBufferToRabbitMQProducer::removeConfirmed(UInt64 received_delivery_tag void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueue> & messages, bool republishing) { std::pair payload; - while (!messages.empty() && producer_channel->usable()) + + /* It is important to make sure that delivery_record.size() is never bigger than returned.size(), i.e. number if unacknowledged + * messages cannot exceed returned.size(), because they all might end up there. + */ + while (!messages.empty() && producer_channel->usable() && delivery_record.size() < RETURNED_LIMIT) { messages.pop(payload); AMQP::Envelope envelope(payload.second.data(), payload.second.size()); - /// if headers exchange - routing keys are added here via headers, else - it is just empty. + /// if headers exchange is used, routing keys are added here via headers, if not - it is just empty. AMQP::Table message_settings = key_arguments; /* There is the case when connection is lost in the period after some messages were published and before ack/nack was sent by the - * server, then it means that publisher will never now whether those messages were delivered or not, and therefore those records + * server, then it means that publisher will never know whether those messages were delivered or not, and therefore those records * that received no ack/nack before connection loss will be republished (see onError() callback), so there might be duplicates. To * let consumer know that received message might be a possible duplicate - a "republished" field is added to message metadata. */ message_settings["republished"] = std::to_string(republishing); - envelope.setHeaders(message_settings); - /* Adding here a message_id property to message metadata. - * (https://stackoverflow.com/questions/59384305/rabbitmq-how-to-handle-unwanted-duplicate-un-ack-message-after-connection-lost) + /* Adding here a messageID property to message metadata. Since RabbitMQ does not guarantee excatly-once delivery, then on the + * consumer side "republished" field of message metadata can be checked and, if it set to 1, consumer might also check "messageID" + * property. This way detection of duplicates is guaranteed. */ - envelope.setMessageID(channel_id + "-" + std::to_string(payload.first)); + envelope.setMessageID(std::to_string(payload.first)); /// Delivery mode is 1 or 2. 1 is default. 2 makes a message durable, but makes performance 1.5-2 times worse. if (persistent) @@ -277,10 +295,11 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueuepublish(exchange_name, routing_keys[0], envelope); } + /// This is needed for "publisher confirms", which guarantees at-least-once delivery. ++delivery_tag; delivery_record.insert(delivery_record.end(), {delivery_tag, payload}); - /// Need to break at some point to let event loop run, because no publishing actually happend before looping. + /// Need to break at some point to let event loop run, because no publishing actually happens before looping. if (delivery_tag % BATCH == 0) break; } @@ -291,33 +310,30 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueueusable()) - publish(returned, true); - else if (!payloads.empty() && producer_channel->usable()) - publish(payloads, false); - } + /* Publish main paylods only when there are no returned messages. This way it is ensured that returned messages are republished + * as fast as possible and no new publishes are made before returned messages are handled. Also once payloads.queue lacks space + * - push attemt will block thread in countRow() - this is intended. + */ + if (!returned.empty() && producer_channel->usable()) + publish(returned, true); + else if (!payloads.empty() && producer_channel->usable()) + publish(payloads, false); iterateEventLoop(); + /* wait_num != 0 if there will be no new payloads pushed to payloads.queue in countRow(), delivery_record is empty if there are + * no more pending acknowldgements from the server (if receieved ack(), records are deleted, if received nack(), records are pushed + * to returned.queue and deleted, because server will attach new delivery tags to them). + */ if (wait_num.load() && delivery_record.empty() && payloads.empty() && returned.empty()) wait_all = false; else if ((!producer_channel->usable() && connection->usable()) || (!connection->usable() && setupConnection(true))) setupChannel(); } - LOG_DEBUG(log, "Processing ended"); + LOG_DEBUG(log, "Prodcuer on channel {} completed", channel_id); } @@ -353,7 +369,7 @@ void WriteBufferToRabbitMQProducer::commit() * RabbitMQ transactions seem not trust-worthy at all - see https://www.rabbitmq.com/semantics.html. Seems like its best to always * use "publisher confirms" rather than transactions (and by default it is so). Probably even need to delete this option. */ - if (!use_tx || !producer_channel->usable()) + if (!use_txn || !producer_channel->usable()) return; std::atomic answer_received = false, wait_rollback = false; diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index 95d505bafd5..85c90cd0ce9 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -25,7 +25,8 @@ public: const String & exchange_name_, const AMQP::ExchangeType exchange_type_, const size_t channel_id_, - const bool use_tx_, + const String channel_base_, + const bool use_txn_, const bool persistent_, Poco::Logger * log_, std::optional delimiter, @@ -55,8 +56,9 @@ private: const Names routing_keys; const String exchange_name; AMQP::ExchangeType exchange_type; - const String channel_id; - const bool use_tx; + const String channel_id_base; + const String channel_base; + const bool use_txn; const bool persistent; AMQP::Table key_arguments; @@ -67,12 +69,14 @@ private: std::unique_ptr connection; std::unique_ptr producer_channel; + String channel_id; ConcurrentBoundedQueue> payloads, returned; UInt64 delivery_tag = 0; std::atomic wait_all = true; std::atomic wait_num = 0; UInt64 payload_counter = 0; std::map> delivery_record; + UInt64 channel_id_counter = 0; Poco::Logger * log; const std::optional delim; diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index a670ea8ab54..bb65319a3be 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -496,7 +496,7 @@ def test_rabbitmq_big_message(rabbitmq_cluster): DROP TABLE test.view; ''') - assert int(result) == rabbitmq_messages*batch_messages, 'ClickHouse lost some messages: {}'.format(result) + assert int(result) == rabbitmq_messages * batch_messages, 'ClickHouse lost some messages: {}'.format(result) @pytest.mark.timeout(420) @@ -516,12 +516,12 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): rabbitmq_row_delimiter = '\\n'; DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.view (key UInt64, value UInt64, consumer_tag String) + CREATE TABLE test.view (key UInt64, value UInt64, channel_id String) ENGINE = MergeTree ORDER BY key SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3; CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT *, _consumer_tag AS consumer_tag FROM test.rabbitmq; + SELECT *, _channel_id AS channel_id FROM test.rabbitmq; ''') i = [0] @@ -541,8 +541,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): for message in messages: current += 1 mes_id = str(current) - channel.basic_publish(exchange='test_sharding', routing_key='', - properties=pika.BasicProperties(message_id=mes_id), body=message) + channel.basic_publish(exchange='test_sharding', routing_key='', properties=pika.BasicProperties(message_id=mes_id), body=message) connection.close() threads = [] @@ -561,7 +560,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): if int(result1) == messages_num * threads_num: break - result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") + result2 = instance.query("SELECT count(DISTINCT channel_id) FROM test.view") for thread in threads: thread.join() @@ -1153,7 +1152,7 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): def test_rabbitmq_hash_exchange(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, consumer_tag String) + CREATE TABLE test.destination(key UInt64, value UInt64, channel_id String) ENGINE = MergeTree() ORDER BY key; ''') @@ -1175,7 +1174,7 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.{0}_mv TO test.destination AS - SELECT key, value, _consumer_tag AS consumer_tag FROM test.{0}; + SELECT key, value, _channel_id AS channel_id FROM test.{0}; '''.format(table_name)) i = [0] @@ -1215,7 +1214,8 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): time.sleep(1) if int(result1) == messages_num * threads_num: break - result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.destination") + + result2 = instance.query("SELECT count(DISTINCT channel_id) FROM test.destination") for consumer_id in range(num_tables): table_name = 'rabbitmq_consumer{}'.format(consumer_id) @@ -1413,7 +1413,7 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): rabbitmq_exchange_name = 'virtuals', rabbitmq_format = 'JSONEachRow'; CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT value, key, _exchange_name, _consumer_tag, _delivery_tag, _redelivered FROM test.rabbitmq_virtuals; + SELECT value, key, _exchange_name, _channel_id, _delivery_tag, _redelivered FROM test.rabbitmq_virtuals; ''') credentials = pika.PlainCredentials('root', 'clickhouse') @@ -1422,11 +1422,11 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): channel = connection.channel() message_num = 10 - i = [0] + i = 0 messages = [] for _ in range(message_num): - messages.append(json.dumps({'key': i[0], 'value': i[0]})) - i[0] += 1 + messages.append(json.dumps({'key': i, 'value': i})) + i += 1 for message in messages: channel.basic_publish(exchange='virtuals', routing_key='', body=message) @@ -1439,33 +1439,28 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): connection.close() - result = instance.query("SELECT count(DISTINCT _delivery_tag) FROM test.view") - assert int(result) == 10 - - result = instance.query("SELECT count(DISTINCT _consumer_tag) FROM test.view") - assert int(result) == 1 - result = instance.query(''' - SELECT key, value, _exchange_name, SUBSTRING(_consumer_tag, 1, 8), _delivery_tag, _redelivered - FROM test.view - ORDER BY key + SELECT key, value, _exchange_name, SUBSTRING(_channel_id, 34, 3), _delivery_tag, _redelivered + FROM test.view ORDER BY key ''') expected = '''\ -0 0 virtuals amq.ctag 1 0 -1 1 virtuals amq.ctag 2 0 -2 2 virtuals amq.ctag 3 0 -3 3 virtuals amq.ctag 4 0 -4 4 virtuals amq.ctag 5 0 -5 5 virtuals amq.ctag 6 0 -6 6 virtuals amq.ctag 7 0 -7 7 virtuals amq.ctag 8 0 -8 8 virtuals amq.ctag 9 0 -9 9 virtuals amq.ctag 10 0 +0 0 virtuals 1_0 1 0 +1 1 virtuals 1_0 2 0 +2 2 virtuals 1_0 3 0 +3 3 virtuals 1_0 4 0 +4 4 virtuals 1_0 5 0 +5 5 virtuals 1_0 6 0 +6 6 virtuals 1_0 7 0 +7 7 virtuals 1_0 8 0 +8 8 virtuals 1_0 9 0 +9 9 virtuals 1_0 10 0 ''' + instance.query(''' DROP TABLE IF EXISTS test.rabbitmq_virtuals_mv ''') + assert TSV(result) == TSV(expected) @@ -1480,10 +1475,10 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): rabbitmq_exchange_name = 'virtuals_mv', rabbitmq_format = 'JSONEachRow'; CREATE TABLE test.view (key UInt64, value UInt64, - exchange_name String, consumer_tag String, delivery_tag UInt64, redelivered UInt8) ENGINE = MergeTree() + exchange_name String, channel_id String, delivery_tag UInt64, redelivered UInt8) ENGINE = MergeTree() ORDER BY key; CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT *, _exchange_name as exchange_name, _consumer_tag as consumer_tag, _delivery_tag as delivery_tag, _redelivered as redelivered + SELECT *, _exchange_name as exchange_name, _channel_id as channel_id, _delivery_tag as delivery_tag, _redelivered as redelivered FROM test.rabbitmq_virtuals_mv; ''') @@ -1493,11 +1488,11 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): channel = connection.channel() message_num = 10 - i = [0] + i = 0 messages = [] for _ in range(message_num): - messages.append(json.dumps({'key': i[0], 'value': i[0]})) - i[0] += 1 + messages.append(json.dumps({'key': i, 'value': i})) + i += 1 for message in messages: channel.basic_publish(exchange='virtuals_mv', routing_key='', body=message) @@ -1510,24 +1505,18 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): connection.close() - result = instance.query("SELECT count(DISTINCT delivery_tag) FROM test.view") - assert int(result) == 10 - - result = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.view") - assert int(result) == 1 - - result = instance.query("SELECT key, value, exchange_name, SUBSTRING(consumer_tag, 1, 8), delivery_tag, redelivered FROM test.view ORDER BY delivery_tag") + result = instance.query("SELECT key, value, exchange_name, SUBSTRING(channel_id, 34, 3), delivery_tag, redelivered FROM test.view ORDER BY delivery_tag") expected = '''\ -0 0 virtuals_mv amq.ctag 1 0 -1 1 virtuals_mv amq.ctag 2 0 -2 2 virtuals_mv amq.ctag 3 0 -3 3 virtuals_mv amq.ctag 4 0 -4 4 virtuals_mv amq.ctag 5 0 -5 5 virtuals_mv amq.ctag 6 0 -6 6 virtuals_mv amq.ctag 7 0 -7 7 virtuals_mv amq.ctag 8 0 -8 8 virtuals_mv amq.ctag 9 0 -9 9 virtuals_mv amq.ctag 10 0 +0 0 virtuals_mv 1_0 1 0 +1 1 virtuals_mv 1_0 2 0 +2 2 virtuals_mv 1_0 3 0 +3 3 virtuals_mv 1_0 4 0 +4 4 virtuals_mv 1_0 5 0 +5 5 virtuals_mv 1_0 6 0 +6 6 virtuals_mv 1_0 7 0 +7 7 virtuals_mv 1_0 8 0 +8 8 virtuals_mv 1_0 9 0 +9 9 virtuals_mv 1_0 10 0 ''' instance.query(''' @@ -1540,7 +1529,7 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): @pytest.mark.timeout(420) -def test_rabbitmq_queue_resume_1(rabbitmq_cluster): +def test_rabbitmq_queue_resume(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) ENGINE = RabbitMQ @@ -1548,54 +1537,6 @@ def test_rabbitmq_queue_resume_1(rabbitmq_cluster): rabbitmq_exchange_name = 'queue_resume', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'queue_resume', - rabbitmq_num_consumers = '2', - rabbitmq_num_queues = '2', - rabbitmq_queue_base = 'queue_resume', - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - ''') - - i = [0] - messages_num = 1000 - - credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) - def produce(): - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({'key': i[0], 'value': i[0]})) - i[0] += 1 - for message in messages: - channel.basic_publish(exchange='queue_resume', routing_key='queue_resume', body=message, - properties=pika.BasicProperties(delivery_mode = 2)) - connection.close() - - threads = [] - threads_num = 10 - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - for thread in threads: - thread.join() - - instance.query(''' - DROP TABLE IF EXISTS test.rabbitmq_queue_resume; - ''') - - instance.query(''' - CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'queue_resume', - rabbitmq_exchange_type = 'direct', - rabbitmq_routing_key_list = 'queue_resume', - rabbitmq_num_consumers = '2', - rabbitmq_num_queues = '2', rabbitmq_queue_base = 'queue_resume', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -1608,42 +1549,6 @@ def test_rabbitmq_queue_resume_1(rabbitmq_cluster): SELECT * FROM test.rabbitmq_queue_resume; ''') - while True: - result1 = instance.query('SELECT count() FROM test.view') - time.sleep(1) - if int(result1) == messages_num * threads_num: - break - - instance.query(''' - DROP TABLE IF EXISTS test.rabbitmq_queue_resume; - DROP TABLE IF EXISTS test.consumer; - DROP TABLE IF EXISTS test.view; - ''') - - assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) - - -@pytest.mark.timeout(420) -def test_rabbitmq_queue_resume_2(rabbitmq_cluster): - instance.query(''' - CREATE TABLE test.rabbitmq_queue_resume (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'queue_resume', - rabbitmq_exchange_type = 'direct', - rabbitmq_routing_key_list = 'queue_resume', - rabbitmq_queue_base = 'queue_resume', - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.view (key UInt64, value UInt64, consumer_tag String) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT *, _consumer_tag AS consumer_tag FROM test.rabbitmq_queue_resume; - ''') - i = [0] messages_num = 10000 @@ -1696,7 +1601,7 @@ def test_rabbitmq_queue_resume_2(rabbitmq_cluster): while True: result1 = instance.query('SELECT count() FROM test.view') time.sleep(1) - if int(result1) == messages_num * threads_num: + if int(result1) >= messages_num * threads_num: break instance.query(''' @@ -1705,59 +1610,44 @@ def test_rabbitmq_queue_resume_2(rabbitmq_cluster): DROP TABLE IF EXISTS test.view; ''') - assert int(result1) == messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result1) >= messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) @pytest.mark.timeout(420) -def test_rabbitmq_consumer_acknowledgements(rabbitmq_cluster): +def test_rabbitmq_no_loss_on_table_drop(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq_consumer_acks (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'consumer_acks', - rabbitmq_exchange_type = 'direct', - rabbitmq_routing_key_list = 'consumer_acks', rabbitmq_queue_base = 'consumer_resume', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; ''') - i = [0] - messages_num = 5000 + i = 0 + messages_num = 100000 credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) - def produce(): - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({'key': i[0], 'value': i[0]})) - i[0] += 1 - for message in messages: - channel.basic_publish(exchange='consumer_acks', routing_key='consumer_acks', body=message, - properties=pika.BasicProperties(delivery_mode = 2)) - connection.close() - - threads = [] - threads_num = 20 - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - for thread in threads: - thread.join() + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + messages = [] + for _ in range(messages_num): + messages.append(json.dumps({'key': i, 'value': i})) + i += 1 + for message in messages: + channel.basic_publish(exchange='consumer_acks', routing_key='', body=message, properties=pika.BasicProperties(delivery_mode = 2)) + connection.close() instance.query(''' DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.view (key UInt64, value UInt64, consumer_tag String) + CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree ORDER BY key; CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT *, _consumer_tag AS consumer_tag FROM test.rabbitmq_consumer_acks; + SELECT * FROM test.rabbitmq_consumer_acks; ''') while int(instance.query('SELECT count() FROM test.view')) == 0: @@ -1779,27 +1669,25 @@ def test_rabbitmq_consumer_acknowledgements(rabbitmq_cluster): ''') while True: - result1 = instance.query('SELECT count() FROM test.view') + result = instance.query('SELECT count(DISTINCT key) FROM test.view') time.sleep(1) - #print("receiived", result1, "collected", collected) - if int(result1) >= messages_num * threads_num: + if int(result) == messages_num: break instance.query(''' - DROP TABLE IF EXISTS test.rabbitmq_consumer_acks; DROP TABLE IF EXISTS test.consumer; DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.rabbitmq_consumer_acks; ''') - # >= because at-least-once - assert int(result1) >= messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result) == messages_num, 'ClickHouse lost some messages: {}'.format(result) @pytest.mark.timeout(420) def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, consumer_tag String) + CREATE TABLE test.destination(key UInt64, value UInt64, channel_id String) ENGINE = MergeTree() ORDER BY key; ''') @@ -1820,7 +1708,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.many_consumers_{0}_mv TO test.destination AS - SELECT key, value, _consumer_tag as consumer_tag FROM test.many_consumers_{0}; + SELECT key, value, _channel_id as channel_id FROM test.many_consumers_{0}; '''.format(table_id)) i = [0] @@ -1860,7 +1748,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): if int(result1) == messages_num * threads_num: break - result2 = instance.query("SELECT count(DISTINCT consumer_tag) FROM test.destination") + result2 = instance.query("SELECT count(DISTINCT channel_id) FROM test.destination") for thread in threads: thread.join() @@ -1881,7 +1769,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): @pytest.mark.timeout(420) -def test_rabbitmq_consumer_restore_connection(rabbitmq_cluster): +def test_rabbitmq_consumer_restore_failed_connection_without_losses(rabbitmq_cluster): instance.query(''' CREATE TABLE test.consumer_reconnect (key UInt64, value UInt64) ENGINE = RabbitMQ @@ -1891,32 +1779,21 @@ def test_rabbitmq_consumer_restore_connection(rabbitmq_cluster): rabbitmq_row_delimiter = '\\n'; ''') - i = [0] - messages_num = 5000 + i = 0 + messages_num = 100000 credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) - def produce(): - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({'key': i[0], 'value': i[0]})) - i[0] += 1 - for message in messages: - channel.basic_publish(exchange='consumer_reconnect', routing_key='', body=message, properties=pika.BasicProperties(delivery_mode = 2)) - connection.close() - threads = [] - threads_num = 20 - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - for thread in threads: - thread.join() + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + messages = [] + for _ in range(messages_num): + messages.append(json.dumps({'key': i, 'value': i})) + i += 1 + for message in messages: + channel.basic_publish(exchange='consumer_reconnect', routing_key='', body=message, properties=pika.BasicProperties(delivery_mode = 2)) + connection.close() instance.query(''' DROP TABLE IF EXISTS test.view; @@ -1929,19 +1806,18 @@ def test_rabbitmq_consumer_restore_connection(rabbitmq_cluster): ''') while int(instance.query('SELECT count() FROM test.view')) == 0: - time.sleep(1) + time.sleep(0.1) kill_rabbitmq(); time.sleep(4); revive_rabbitmq(); - collected = int(instance.query('SELECT count() FROM test.view')) + #collected = int(instance.query('SELECT count() FROM test.view')) while True: - result = instance.query('SELECT count() FROM test.view') + result = instance.query('SELECT count(DISTINCT key) FROM test.view') time.sleep(1) - print("receiived", result, "collected", collected) - if int(result) >= messages_num * threads_num: + if int(result) == messages_num: break instance.query(''' @@ -1950,33 +1826,26 @@ def test_rabbitmq_consumer_restore_connection(rabbitmq_cluster): DROP TABLE IF EXISTS test.consumer_reconnect; ''') - # >= because at-least-once - assert int(result) >= messages_num * threads_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result) == messages_num, 'ClickHouse lost some messages: {}'.format(result) @pytest.mark.timeout(420) -def test_rabbitmq_producer_restore_connection(rabbitmq_cluster): - instance.query(''' - DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - ''') - +def test_rabbitmq_producer_restore_failed_connection_without_losses(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.consume; - DROP TABLE IF EXISTS test.consume_mv; + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree + ORDER BY key; CREATE TABLE test.consume (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'producer_reconnect', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; - CREATE MATERIALIZED VIEW test.consume_mv TO test.destination AS - SELECT key, value FROM test.consume; - ''') - - instance.query(''' + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.consume; DROP TABLE IF EXISTS test.producer_reconnect; CREATE TABLE test.producer_reconnect (key UInt64, value UInt64) ENGINE = RabbitMQ @@ -2008,7 +1877,7 @@ def test_rabbitmq_producer_restore_connection(rabbitmq_cluster): else: raise - while int(instance.query('SELECT count() FROM test.destination')) == 0: + while int(instance.query('SELECT count() FROM test.view')) == 0: time.sleep(0.1) kill_rabbitmq(); @@ -2016,20 +1885,87 @@ def test_rabbitmq_producer_restore_connection(rabbitmq_cluster): revive_rabbitmq(); while True: - result = instance.query('SELECT count() FROM test.destination') + result = instance.query('SELECT count(DISTINCT key) FROM test.view') time.sleep(1) - print(result, messages_num) - if int(result) >= messages_num: + if int(result) == messages_num: break instance.query(''' - DROP TABLE IF EXISTS test.consume_mv; + DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consume; DROP TABLE IF EXISTS test.producer_reconnect; - DROP TABLE IF EXISTS test.destination; ''') - assert int(result) >= messages_num, 'ClickHouse lost some messages: {}'.format(result) + assert int(result) == messages_num, 'ClickHouse lost some messages: {}'.format(result) + + +@pytest.mark.timeout(420) +def test_rabbitmq_virtual_columns_2(rabbitmq_cluster): + instance.query(''' + DROP TABLE IF EXISTS test.destination; + CREATE TABLE test.destination(key UInt64, value UInt64, + exchange_name String, channel_id String, delivery_tag UInt64, redelivered UInt8) ENGINE = MergeTree() + ORDER BY key; + ''') + + table_num = 3 + for table_id in range(table_num): + print("Setting up table {}".format(table_id)) + instance.query(''' + DROP TABLE IF EXISTS test.virtuals_{0}; + DROP TABLE IF EXISTS test.virtuals_{0}_mv; + CREATE TABLE test.virtuals_{0} (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'virtuals_2', + rabbitmq_num_queues = 2, + rabbitmq_num_consumers = 2, + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; + CREATE MATERIALIZED VIEW test.many_consumers_{0}_mv TO test.destination AS + SELECT *, _exchange_name as exchange_name, _channel_id as channel_id, _delivery_tag as delivery_tag, _redelivered as redelivered + FROM test.virtuals_{0}; + '''.format(table_id)) + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + + message_num = 10 + i = 0 + messages = [] + for _ in range(message_num): + messages.append(json.dumps({'key': i, 'value': i})) + i += 1 + + for i in range(message_num): + channel.basic_publish(exchange='virtuals_2', routing_key='', body=messages[i], + properties=pika.BasicProperties(delivery_mode=2, message_id=str(i))) + + #kill_rabbitmq(); + #time.sleep(2); + #revive_rabbitmq(); + + while True: + result = instance.query('SELECT count(DISTINCT concat([channel_id], [toString(delivery_tag)])) FROM test.destination') + print instance.query(''' + SELECT DISTINCT concat([channel_id], [toString(delivery_tag)]) + FROM (SELECT channel_id AS id, delivery_tag AS tag FROM test.destination GROUP BY id ORDER BY tag)''') + time.sleep(1) + if int(result) == message_num * table_num: + break + + connection.close() + + instance.query(''' + DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.rabbitmq_virtuals_mv + ''') + + assert int(result) == message_num * table_num if __name__ == '__main__': From 1006c4f11bce91b4fdf82f575f8f427828347805 Mon Sep 17 00:00:00 2001 From: "Ivan A. Torgashov" Date: Sat, 15 Aug 2020 14:18:17 +0500 Subject: [PATCH 0054/1911] Update tests for Redis dictionary requirepass authorization support --- .../integration/runner/compose/docker_compose_redis.yml | 1 + .../external_sources.py | 9 ++++++--- .../test_dictionaries_all_layouts_and_sources/test.py | 4 ++-- 3 files changed, 9 insertions(+), 5 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_redis.yml b/docker/test/integration/runner/compose/docker_compose_redis.yml index 2dc79ed5910..2c9ace96d0c 100644 --- a/docker/test/integration/runner/compose/docker_compose_redis.yml +++ b/docker/test/integration/runner/compose/docker_compose_redis.yml @@ -5,3 +5,4 @@ services: restart: always ports: - 6380:6379 + command: redis-server --requirepass "clickhouse" diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/external_sources.py b/tests/integration/test_dictionaries_all_layouts_and_sources/external_sources.py index f6985e7de54..fac7dcdea1e 100644 --- a/tests/integration/test_dictionaries_all_layouts_and_sources/external_sources.py +++ b/tests/integration/test_dictionaries_all_layouts_and_sources/external_sources.py @@ -483,23 +483,27 @@ class SourceRedis(ExternalSource): name, internal_hostname, internal_port, docker_hostname, docker_port, user, password ) self.storage_type = storage_type + self.db_index = 1 def get_source_str(self, table_name): return ''' {host} {port} - 0 + {password} + {db_index} {storage_type} '''.format( host=self.docker_hostname, port=self.docker_port, + password=self.password, storage_type=self.storage_type, # simple or hash_map + db_index=self.db_index, ) def prepare(self, structure, table_name, cluster): - self.client = redis.StrictRedis(host=self.internal_hostname, port=self.internal_port) + self.client = redis.StrictRedis(host=self.internal_hostname, port=self.internal_port, db=self.db_index, password=self.password or None) self.prepared = True self.ordered_names = structure.get_ordered_names() @@ -525,7 +529,6 @@ class SourceRedis(ExternalSource): return True return False - class SourceAerospike(ExternalSource): def __init__(self, name, internal_hostname, internal_port, docker_hostname, docker_port, user, password): diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/test.py b/tests/integration/test_dictionaries_all_layouts_and_sources/test.py index f4b0ba9c1e4..994d8e5e65d 100644 --- a/tests/integration/test_dictionaries_all_layouts_and_sources/test.py +++ b/tests/integration/test_dictionaries_all_layouts_and_sources/test.py @@ -134,8 +134,8 @@ DICTIONARIES = [] # Key-value dictionaries with only one possible field for key SOURCES_KV = [ - SourceRedis("RedisSimple", "localhost", "6380", "redis1", "6379", "", "", storage_type="simple"), - SourceRedis("RedisHash", "localhost", "6380", "redis1", "6379", "", "", storage_type="hash_map"), + SourceRedis("RedisSimple", "localhost", "6380", "redis1", "6379", "", "clickhouse", storage_type="simple"), + SourceRedis("RedisHash", "localhost", "6380", "redis1", "6379", "", "clickhouse", storage_type="hash_map"), ] DICTIONARIES_KV = [] From 43839a97b6a214cdbeeb5d6fdbf8c9cccfbd5e95 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 15 Aug 2020 21:29:24 +0800 Subject: [PATCH 0055/1911] ISSUES-4006 add factor with DateTime type --- src/DataTypes/DataTypeDateTime.cpp | 27 ----- src/DataTypes/DataTypeDateTime64.cpp | 61 ---------- src/DataTypes/registerDataTypeDateTime.cpp | 110 ++++++++++++++++++ src/DataTypes/ya.make | 1 + src/Functions/FunctionsConversion.cpp | 1 + src/Functions/FunctionsConversion.h | 33 ++++++ .../01442_date_time_with_params.reference | 4 + .../01442_date_time_with_params.sql | 15 +++ 8 files changed, 164 insertions(+), 88 deletions(-) create mode 100644 src/DataTypes/registerDataTypeDateTime.cpp create mode 100644 tests/queries/0_stateless/01442_date_time_with_params.reference create mode 100644 tests/queries/0_stateless/01442_date_time_with_params.sql diff --git a/src/DataTypes/DataTypeDateTime.cpp b/src/DataTypes/DataTypeDateTime.cpp index c860766406e..9ea698d4fbb 100644 --- a/src/DataTypes/DataTypeDateTime.cpp +++ b/src/DataTypes/DataTypeDateTime.cpp @@ -185,31 +185,4 @@ bool DataTypeDateTime::equals(const IDataType & rhs) const return typeid(rhs) == typeid(*this); } -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; -} - -static DataTypePtr create(const ASTPtr & arguments) -{ - if (!arguments) - return std::make_shared(); - - if (arguments->children.size() != 1) - throw Exception("DateTime data type can optionally have only one argument - time zone name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - const auto * arg = arguments->children[0]->as(); - if (!arg || arg->value.getType() != Field::Types::String) - throw Exception("Parameter for DateTime data type must be string literal", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(arg->value.get()); -} - -void registerDataTypeDateTime(DataTypeFactory & factory) -{ - factory.registerDataType("DateTime", create, DataTypeFactory::CaseInsensitive); - factory.registerAlias("TIMESTAMP", "DateTime", DataTypeFactory::CaseInsensitive); -} - } diff --git a/src/DataTypes/DataTypeDateTime64.cpp b/src/DataTypes/DataTypeDateTime64.cpp index 97dd28439d7..ee4139c2b7a 100644 --- a/src/DataTypes/DataTypeDateTime64.cpp +++ b/src/DataTypes/DataTypeDateTime64.cpp @@ -201,65 +201,4 @@ bool DataTypeDateTime64::equals(const IDataType & rhs) const return false; } -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; -} - -enum class ArgumentKind -{ - Optional, - Mandatory -}; - -template -std::conditional_t, T> -getArgument(const ASTPtr & arguments, size_t argument_index, const char * argument_name, const std::string context_data_type_name) -{ - using NearestResultType = NearestFieldType; - const auto field_type = Field::TypeToEnum::value; - const ASTLiteral * argument = nullptr; - - auto exception_message = [=](const String & message) - { - return std::string("Parameter #") + std::to_string(argument_index) + " '" - + argument_name + "' for " + context_data_type_name - + message - + ", expected: " + Field::Types::toString(field_type) + " literal."; - }; - - if (!arguments || arguments->children.size() <= argument_index - || !(argument = arguments->children[argument_index]->as())) - { - if constexpr (Kind == ArgumentKind::Optional) - return {}; - else - throw Exception(exception_message(" is missing"), - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } - - if (argument->value.getType() != field_type) - throw Exception(exception_message(String(" has wrong type: ") + argument->value.getTypeName()), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return argument->value.get(); -} - -static DataTypePtr create64(const ASTPtr & arguments) -{ - if (!arguments || arguments->size() == 0) - return std::make_shared(DataTypeDateTime64::default_scale); - - const auto scale = getArgument(arguments, 0, "scale", "DateType64"); - const auto timezone = getArgument(arguments, !!scale, "timezone", "DateType64"); - - return std::make_shared(scale.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{})); -} - -void registerDataTypeDateTime64(DataTypeFactory & factory) -{ - factory.registerDataType("DateTime64", create64, DataTypeFactory::CaseInsensitive); -} - } diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp new file mode 100644 index 00000000000..47487641e09 --- /dev/null +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -0,0 +1,110 @@ + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +enum class ArgumentKind +{ + Optional, + Mandatory +}; + +template +std::conditional_t, T> +getArgument(const ASTPtr & arguments, size_t argument_index, const char * argument_name, const std::string context_data_type_name) +{ + using NearestResultType = NearestFieldType; + const auto field_type = Field::TypeToEnum::value; + const ASTLiteral * argument = nullptr; + + auto exception_message = [=](const String & message) + { + return std::string("Parameter #") + std::to_string(argument_index) + " '" + + argument_name + "' for " + context_data_type_name + + message + + ", expected: " + Field::Types::toString(field_type) + " literal."; + }; + + if (!arguments || arguments->children.size() <= argument_index + || !(argument = arguments->children[argument_index]->as()) + || argument->value.getType() != field_type) + { + if constexpr (Kind == ArgumentKind::Optional) + return {}; + else + throw Exception(exception_message(" is missing"), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + return argument->value.get(); +} + +static DataTypePtr create(const ASTPtr & arguments) +{ + if (!arguments || arguments->size() == 0) + return std::make_shared(); + + const auto scale = getArgument(arguments, 0, "scale", "DateTime"); + const auto timezone = getArgument(arguments, !!scale, "timezone", "DateTime"); + + if (scale) + return std::make_shared(scale.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{})); + + return std::make_shared(timezone.value_or(String{})); +} + +static DataTypePtr create32(const ASTPtr & arguments) +{ + if (!arguments || arguments->size() == 0) + return std::make_shared(); + + if (arguments->children.size() != 1) + throw Exception("DateTime32 data type can optionally have only one argument - time zone name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const auto timezone = getArgument(arguments, 0, "timezone", "DateTime32"); + + return std::make_shared(timezone); +} + +static DataTypePtr create64(const ASTPtr & arguments) +{ + if (!arguments || arguments->size() == 0) + return std::make_shared(DataTypeDateTime64::default_scale); + + if (arguments->children.size() > 2) + throw Exception("DateTime64 data type can optionally have two argument - scale and time zone name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const auto scale = getArgument(arguments, 0, "scale", "DateTime64"); + const auto timezone = getArgument(arguments, !!scale, "timezone", "DateTime64"); + + return std::make_shared(scale.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{})); +} + +void registerDataTypeDateTime(DataTypeFactory & factory) +{ + factory.registerDataType("DateTime", create, DataTypeFactory::CaseInsensitive); + factory.registerDataType("DateTime32", create32, DataTypeFactory::CaseInsensitive); + factory.registerDataType("DateTime64", create64, DataTypeFactory::CaseInsensitive); + + factory.registerAlias("TIMESTAMP", "DateTime", DataTypeFactory::CaseInsensitive); +} + +void registerDataTypeDateTime64(DataTypeFactory & /*factory*/) +{ +// factory.registerDataType("DateTime64", create64, DataTypeFactory::CaseInsensitive); +} + +} diff --git a/src/DataTypes/ya.make b/src/DataTypes/ya.make index 82e9baf76f2..4237ca920ae 100644 --- a/src/DataTypes/ya.make +++ b/src/DataTypes/ya.make @@ -38,6 +38,7 @@ SRCS( getMostSubtype.cpp IDataType.cpp NestedUtils.cpp + registerDataTypeDateTime.cpp ) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index da42c8a2623..804c16d946d 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -32,6 +32,7 @@ void registerFunctionsConversion(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 4aacafafd96..a8e8ad81ff8 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -968,6 +968,7 @@ struct ConvertImpl /// Declared early because used below. struct NameToDate { static constexpr auto name = "toDate"; }; struct NameToDateTime { static constexpr auto name = "toDateTime"; }; +struct NameToDateTime32 { static constexpr auto name = "toDateTime32"; }; struct NameToDateTime64 { static constexpr auto name = "toDateTime64"; }; struct NameToString { static constexpr auto name = "toString"; }; struct NameToDecimal32 { static constexpr auto name = "toDecimal32"; }; @@ -1027,6 +1028,14 @@ public: { mandatory_args.push_back({"scale", &isNativeInteger, &isColumnConst, "const Integer"}); } + + if constexpr (std::is_same_v && std::is_same_v) + { + /// toDateTime(value, scale:Integer) + if ((arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3) + mandatory_args.push_back({"scale", &isNativeInteger, &isColumnConst, "const Integer"}); + } + // toString(DateTime or DateTime64, [timezone: String]) if ((std::is_same_v && arguments.size() > 0 && (isDateTime64(arguments[0].type) || isDateTime(arguments[0].type))) // toUnixTimestamp(value[, timezone : String]) @@ -1076,6 +1085,17 @@ public: scale = static_cast(arguments[1].column->get64(0)); } + if constexpr (std::is_same_v && std::is_same_v) + { + /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 + if ((arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3) + { + timezone_arg_position += 1; + scale = static_cast(arguments[1].column->get64(0)); + return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); + } + } + if constexpr (std::is_same_v) return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); else if constexpr (to_datetime64) @@ -1179,6 +1199,18 @@ private: return true; }; + if constexpr (std::is_same_v && std::is_same_v) + { + /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 + if ((arguments.size() == 2 && isUnsignedInteger(block.getByPosition(arguments[1]).type)) || arguments.size() == 3) + { + if (!callOnIndexAndDataType(from_type->getTypeId(), call)) + throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + return; + } + } + bool done = callOnIndexAndDataType(from_type->getTypeId(), call); if (!done) { @@ -1607,6 +1639,7 @@ using FunctionToFloat32 = FunctionConvert>; using FunctionToDate = FunctionConvert; using FunctionToDateTime = FunctionConvert; +using FunctionToDateTime32 = FunctionConvert; using FunctionToDateTime64 = FunctionConvert; using FunctionToUUID = FunctionConvert>; using FunctionToString = FunctionConvert; diff --git a/tests/queries/0_stateless/01442_date_time_with_params.reference b/tests/queries/0_stateless/01442_date_time_with_params.reference new file mode 100644 index 00000000000..a6cb7f7b948 --- /dev/null +++ b/tests/queries/0_stateless/01442_date_time_with_params.reference @@ -0,0 +1,4 @@ +2020-01-01 00:00:00 DateTime 2020-01-01 00:01:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') 2020-01-01 00:05:00 DateTime 2020-01-01 00:06:00 DateTime(\'Europe/Moscow\') +2020-01-01 00:00:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') +2020-01-01 00:00:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') +2020-01-01 00:00:00 DateTime diff --git a/tests/queries/0_stateless/01442_date_time_with_params.sql b/tests/queries/0_stateless/01442_date_time_with_params.sql new file mode 100644 index 00000000000..1e75089bc05 --- /dev/null +++ b/tests/queries/0_stateless/01442_date_time_with_params.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test; + +CREATE TABLE test (a DateTime, b DateTime(), c DateTime(2), d DateTime('Europe/Moscow'), e DateTime(3, 'Europe/Moscow'), f DateTime32, g DateTime32('Europe/Moscow')) ENGINE = MergeTree ORDER BY a; + +INSERT INTO test VALUES('2020-01-01 00:00:00', '2020-01-01 00:01:00', '2020-01-01 00:02:00.11', '2020-01-01 00:03:00', '2020-01-01 00:04:00.22', '2020-01-01 00:05:00', '2020-01-01 00:06:00') + +SELECT a, toTypeName(a), b, toTypeName(b), c, toTypeName(c), d, toTypeName(d), e, toTypeName(e), f, toTypeName(f), g, toTypeName(g) FROM test; + +SELECT toDateTime('2020-01-01 00:00:00') AS a, toTypeName(a), toDateTime('2020-01-01 00:02:00.11', 2) AS b, toTypeName(b), toDateTime('2020-01-01 00:03:00', 'Europe/Moscow') AS c, toTypeName(c), toDateTime('2020-01-01 00:04:00.22', 3, 'Europe/Moscow') AS d, toTypeName(d); + +SELECT CAST('2020-01-01 00:00:00', 'DateTime') AS a, toTypeName(a), CAST('2020-01-01 00:02:00.11', 'DateTime(2)') AS b, toTypeName(b), CAST('2020-01-01 00:03:00', 'DateTime(\'Europe/Moscow\')') AS c, toTypeName(c), CAST('2020-01-01 00:04:00.22', 'DateTime(3, \'Europe/Moscow\')') AS d, toTypeName(d); + +SELECT toDateTime32('2020-01-01 00:00:00') AS a, toTypeName(a); + +DROP TABLE IF EXISTS test; From 4ad267571e5454349e3fca00c9ec34d0c578e794 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 15 Aug 2020 21:43:44 +0800 Subject: [PATCH 0056/1911] ISSUES-4006 remove unused code --- src/DataTypes/DataTypeFactory.cpp | 1 - src/DataTypes/DataTypeFactory.h | 1 - src/DataTypes/registerDataTypeDateTime.cpp | 5 ----- 3 files changed, 7 deletions(-) diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 664927389b5..9386f4b39f1 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -165,7 +165,6 @@ DataTypeFactory::DataTypeFactory() registerDataTypeDecimal(*this); registerDataTypeDate(*this); registerDataTypeDateTime(*this); - registerDataTypeDateTime64(*this); registerDataTypeString(*this); registerDataTypeFixedString(*this); registerDataTypeEnum(*this); diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index 67b72945acc..ea77c50170c 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -82,7 +82,6 @@ void registerDataTypeInterval(DataTypeFactory & factory); void registerDataTypeLowCardinality(DataTypeFactory & factory); void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory); void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory); -void registerDataTypeDateTime64(DataTypeFactory & factory); void registerDataTypeDomainGeo(DataTypeFactory & factory); } diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index 47487641e09..c6a79e48335 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -102,9 +102,4 @@ void registerDataTypeDateTime(DataTypeFactory & factory) factory.registerAlias("TIMESTAMP", "DateTime", DataTypeFactory::CaseInsensitive); } -void registerDataTypeDateTime64(DataTypeFactory & /*factory*/) -{ -// factory.registerDataType("DateTime64", create64, DataTypeFactory::CaseInsensitive); -} - } From 1e7eb494812ffb48f77e6cb58e1f9f323050aa11 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 15 Aug 2020 14:38:29 +0000 Subject: [PATCH 0057/1911] Add one more mes property, support format_schema --- .../RabbitMQ/RabbitMQBlockInputStream.cpp | 6 +- src/Storages/RabbitMQ/RabbitMQSettings.h | 1 + .../ReadBufferFromRabbitMQConsumer.cpp | 18 ++- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 2 + src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 65 +++++--- src/Storages/RabbitMQ/StorageRabbitMQ.h | 3 + .../WriteBufferToRabbitMQProducer.cpp | 2 +- .../format_schemas/rabbitmq.proto | 6 + .../test_storage_rabbitmq/rabbitmq_pb2.py | 77 +++++++++ .../integration/test_storage_rabbitmq/test.py | 151 ++++++++++++------ 10 files changed, 257 insertions(+), 74 deletions(-) create mode 100644 tests/integration/test_storage_rabbitmq/clickhouse_path/format_schemas/rabbitmq.proto create mode 100644 tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py diff --git a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp index e26645a1168..16ba14094ac 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp @@ -22,8 +22,10 @@ RabbitMQBlockInputStream::RabbitMQBlockInputStream( , column_names(columns) , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) , virtual_header(metadata_snapshot->getSampleBlockForColumns( - {"_exchange_name", "_channel_id", "_delivery_tag", "_redelivered"}, storage.getVirtuals(), storage.getStorageID())) + {"_exchange_name", "_channel_id", "_delivery_tag", "_redelivered", "_message_id"}, storage.getVirtuals(), storage.getStorageID())) { + if (!storage.getSchemaName().empty()) + context.setSetting("format_schema", storage.getSchemaName()); } @@ -131,6 +133,7 @@ Block RabbitMQBlockInputStream::readImpl() auto channel_id = buffer->getChannelID(); auto delivery_tag = buffer->getDeliveryTag(); auto redelivered = buffer->getRedelivered(); + auto message_id = buffer->getMessageID(); buffer->updateAckTracker({delivery_tag, channel_id}); @@ -140,6 +143,7 @@ Block RabbitMQBlockInputStream::readImpl() virtual_columns[1]->insert(channel_id); virtual_columns[2]->insert(delivery_tag); virtual_columns[3]->insert(redelivered); + virtual_columns[4]->insert(message_id); } total_rows = total_rows + new_rows; diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index 5d15bd5b77d..2416a15f65a 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -14,6 +14,7 @@ namespace DB M(String, rabbitmq_exchange_name, "clickhouse-exchange", "The exchange name, to which messages are sent.", 0) \ M(String, rabbitmq_format, "", "The message format.", 0) \ M(Char, rabbitmq_row_delimiter, '\0', "The character to be considered as a delimiter.", 0) \ + M(String, rabbitmq_schema, "", "Schema identifier (used by schema-based formats) for RabbitMQ engine", 0) \ M(String, rabbitmq_exchange_type, "default", "The exchange type.", 0) \ M(UInt64, rabbitmq_num_consumers, 1, "The number of consumer channels per table.", 0) \ M(UInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \ diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 833382f354b..197b9f7e057 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -14,6 +14,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + static const auto QUEUE_SIZE = 50000; ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( @@ -51,7 +56,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( consumer_channel->onReady([&]() { - channel_id = channel_base + "_" + std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++); + channel_id = std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++) + "_" + channel_base; LOG_TRACE(log, "Channel {} is created", channel_id); consumer_channel->onError([&](const char * message) @@ -142,7 +147,10 @@ void ReadBufferFromRabbitMQConsumer::subscribe() if (row_delimiter != '\0') message_received += row_delimiter; - received.push({message_received, redelivered, AckTracker(delivery_tag, channel_id)}); + if (message.hasMessageID()) + received.push({message_received, message.messageID(), redelivered, AckTracker(delivery_tag, channel_id)}); + else + received.push({message_received, "", redelivered, AckTracker(delivery_tag, channel_id)}); } }) .onError([&](const char * message) @@ -195,7 +203,11 @@ void ReadBufferFromRabbitMQConsumer::restoreChannel(ChannelPtr new_channel) consumer_channel = std::move(new_channel); consumer_channel->onReady([&]() { - channel_id = channel_base + "_" + std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++); + /* First number indicates current consumer buffer; second number indicates serial number of created channel for current buffer, + * i.e. if channel fails - another one is created and its serial number is incremented; channel_base is to guarantee that + * channel_id is unique for each table. + */ + channel_id = std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++) + "_" + channel_base; LOG_TRACE(log, "Channel {} is created", channel_id); consumer_channel->onError([&](const char * message) diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index c5643cb59f4..e00e8172509 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -51,6 +51,7 @@ public: struct MessageData { String message; + String message_id; bool redelivered; AckTracker track; }; @@ -65,6 +66,7 @@ public: auto getChannelID() const { return current.track.channel_id; } auto getDeliveryTag() const { return current.track.delivery_tag; } auto getRedelivered() const { return current.redelivered; } + auto getMessageID() const { return current.message_id; } private: bool nextImpl() override; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index f85f7d6b59c..f82773ed367 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -69,6 +69,7 @@ StorageRabbitMQ::StorageRabbitMQ( const String & exchange_name_, const String & format_name_, char row_delimiter_, + const String & schema_name_, const String & exchange_type_, size_t num_consumers_, size_t num_queues_, @@ -83,6 +84,7 @@ StorageRabbitMQ::StorageRabbitMQ( , exchange_name(exchange_name_) , format_name(global_context.getMacros()->expand(format_name_)) , row_delimiter(row_delimiter_) + , schema_name(global_context.getMacros()->expand(schema_name_)) , num_consumers(num_consumers_) , num_queues(num_queues_) , use_transactional_channel(use_transactional_channel_) @@ -785,13 +787,29 @@ void registerStorageRabbitMQ(StorageFactory & factory) } } - String exchange_type = rabbitmq_settings.rabbitmq_exchange_type.value; + String schema = rabbitmq_settings.rabbitmq_schema.value; if (args_count >= 6) { engine_args[5] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[5], args.local_context); const auto * ast = engine_args[5]->as(); if (ast && ast->value.getType() == Field::Types::String) + { + schema = safeGet(ast->value); + } + else + { + throw Exception("Format schema must be a string", ErrorCodes::BAD_ARGUMENTS); + } + } + + String exchange_type = rabbitmq_settings.rabbitmq_exchange_type.value; + if (args_count >= 7) + { + engine_args[6] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[6], args.local_context); + + const auto * ast = engine_args[6]->as(); + if (ast && ast->value.getType() == Field::Types::String) { exchange_type = safeGet(ast->value); } @@ -802,9 +820,9 @@ void registerStorageRabbitMQ(StorageFactory & factory) } UInt64 num_consumers = rabbitmq_settings.rabbitmq_num_consumers; - if (args_count >= 7) + if (args_count >= 8) { - const auto * ast = engine_args[6]->as(); + const auto * ast = engine_args[7]->as(); if (ast && ast->value.getType() == Field::Types::UInt64) { num_consumers = safeGet(ast->value); @@ -816,9 +834,9 @@ void registerStorageRabbitMQ(StorageFactory & factory) } UInt64 num_queues = rabbitmq_settings.rabbitmq_num_queues; - if (args_count >= 8) + if (args_count >= 9) { - const auto * ast = engine_args[7]->as(); + const auto * ast = engine_args[8]->as(); if (ast && ast->value.getType() == Field::Types::UInt64) { num_consumers = safeGet(ast->value); @@ -830,9 +848,9 @@ void registerStorageRabbitMQ(StorageFactory & factory) } bool use_transactional_channel = static_cast(rabbitmq_settings.rabbitmq_transactional_channel); - if (args_count >= 9) + if (args_count >= 10) { - const auto * ast = engine_args[8]->as(); + const auto * ast = engine_args[9]->as(); if (ast && ast->value.getType() == Field::Types::UInt64) { use_transactional_channel = static_cast(safeGet(ast->value)); @@ -844,33 +862,33 @@ void registerStorageRabbitMQ(StorageFactory & factory) } String queue_base = rabbitmq_settings.rabbitmq_queue_base.value; - if (args_count >= 10) - { - engine_args[9] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[9], args.local_context); - - const auto * ast = engine_args[9]->as(); - if (ast && ast->value.getType() == Field::Types::String) - { - queue_base = safeGet(ast->value); - } - } - - String deadletter_exchange = rabbitmq_settings.rabbitmq_deadletter_exchange.value; if (args_count >= 11) { engine_args[10] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[10], args.local_context); const auto * ast = engine_args[10]->as(); if (ast && ast->value.getType() == Field::Types::String) + { + queue_base = safeGet(ast->value); + } + } + + String deadletter_exchange = rabbitmq_settings.rabbitmq_deadletter_exchange.value; + if (args_count >= 12) + { + engine_args[11] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[11], args.local_context); + + const auto * ast = engine_args[11]->as(); + if (ast && ast->value.getType() == Field::Types::String) { deadletter_exchange = safeGet(ast->value); } } bool persistent = static_cast(rabbitmq_settings.rabbitmq_persistent_mode); - if (args_count >= 12) + if (args_count >= 13) { - const auto * ast = engine_args[11]->as(); + const auto * ast = engine_args[12]->as(); if (ast && ast->value.getType() == Field::Types::UInt64) { persistent = static_cast(safeGet(ast->value)); @@ -883,7 +901,7 @@ void registerStorageRabbitMQ(StorageFactory & factory) return StorageRabbitMQ::create( args.table_id, args.context, args.columns, - host_port, routing_keys, exchange, format, row_delimiter, exchange_type, num_consumers, + host_port, routing_keys, exchange, format, row_delimiter, schema, exchange_type, num_consumers, num_queues, use_transactional_channel, queue_base, deadletter_exchange, persistent); }; @@ -898,7 +916,8 @@ NamesAndTypesList StorageRabbitMQ::getVirtuals() const {"_exchange_name", std::make_shared()}, {"_channel_id", std::make_shared()}, {"_delivery_tag", std::make_shared()}, - {"_redelivered", std::make_shared()} + {"_redelivered", std::make_shared()}, + {"_message_id", std::make_shared()} }; } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 0960e35d3bf..60bc1aa7157 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -55,6 +55,7 @@ public: const String & getFormatName() const { return format_name; } NamesAndTypesList getVirtuals() const override; + const auto & getSchemaName() const { return schema_name; } const String getExchange() const { return exchange_name; } bool checkBridge() const { return !exchange_removed.load(); } @@ -74,6 +75,7 @@ protected: const String & exchange_name_, const String & format_name_, char row_delimiter_, + const String & schema_name_, const String & exchange_type_, size_t num_consumers_, size_t num_queues_, @@ -92,6 +94,7 @@ private: const String format_name; char row_delimiter; + const String schema_name; size_t num_consumers; size_t num_created_consumers = 0; bool hash_exchange; diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 2b818f0341f..8cd769e792f 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -186,7 +186,7 @@ void WriteBufferToRabbitMQProducer::setupChannel() producer_channel->onReady([&]() { - channel_id = channel_base + "_" + channel_id_base + std::to_string(channel_id_counter++); + channel_id = channel_id_base + std::to_string(channel_id_counter++) + "_" + channel_base; LOG_DEBUG(log, "Producer's channel {} is ready", channel_id); if (use_txn) diff --git a/tests/integration/test_storage_rabbitmq/clickhouse_path/format_schemas/rabbitmq.proto b/tests/integration/test_storage_rabbitmq/clickhouse_path/format_schemas/rabbitmq.proto new file mode 100644 index 00000000000..96b24be4938 --- /dev/null +++ b/tests/integration/test_storage_rabbitmq/clickhouse_path/format_schemas/rabbitmq.proto @@ -0,0 +1,6 @@ +syntax = "proto3"; + +message KeyValuePair { + uint64 key = 1; + string value = 2; +} \ No newline at end of file diff --git a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py new file mode 100644 index 00000000000..fb0f1413eac --- /dev/null +++ b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py @@ -0,0 +1,77 @@ +# -*- coding: utf-8 -*- +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: clickhouse_path/format_schemas/rabbitmq.proto +"""Generated protocol buffer code.""" +from google.protobuf import descriptor as _descriptor +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + + + +DESCRIPTOR = _descriptor.FileDescriptor( + name='clickhouse_path/format_schemas/rabbitmq.proto', + package='', + syntax='proto3', + serialized_options=None, + create_key=_descriptor._internal_create_key, + serialized_pb=b'\n-clickhouse_path/format_schemas/rabbitmq.proto\"*\n\x0cKeyValuePair\x12\x0b\n\x03key\x18\x01 \x01(\x04\x12\r\n\x05value\x18\x02 \x01(\tb\x06proto3' +) + + + + +_KEYVALUEPAIR = _descriptor.Descriptor( + name='KeyValuePair', + full_name='KeyValuePair', + filename=None, + file=DESCRIPTOR, + containing_type=None, + create_key=_descriptor._internal_create_key, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='KeyValuePair.key', index=0, + number=1, type=4, cpp_type=4, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key), + _descriptor.FieldDescriptor( + name='value', full_name='KeyValuePair.value', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=b"".decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=49, + serialized_end=91, +) + +DESCRIPTOR.message_types_by_name['KeyValuePair'] = _KEYVALUEPAIR +_sym_db.RegisterFileDescriptor(DESCRIPTOR) + +KeyValuePair = _reflection.GeneratedProtocolMessageType('KeyValuePair', (_message.Message,), { + 'DESCRIPTOR' : _KEYVALUEPAIR, + '__module__' : 'clickhouse_path.format_schemas.rabbitmq_pb2' + # @@protoc_insertion_point(class_scope:KeyValuePair) + }) +_sym_db.RegisterMessage(KeyValuePair) + + +# @@protoc_insertion_point(module_scope) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index bb65319a3be..b8ccbf9ce56 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -16,13 +16,19 @@ from helpers.network import PartitionManager import json import subprocess +import avro.schema +from confluent.schemaregistry.client import CachedSchemaRegistryClient +from confluent.schemaregistry.serializers.MessageSerializer import MessageSerializer from google.protobuf.internal.encoder import _VarintBytes +import rabbitmq_pb2 + cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', config_dir='configs', main_configs=['configs/rabbitmq.xml','configs/log_conf.xml'], - with_rabbitmq=True) + with_rabbitmq=True, + clickhouse_path_dir='clickhouse_path') rabbitmq_id = '' @@ -316,6 +322,57 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): rabbitmq_check_result(result, True) +@pytest.mark.timeout(180) +def test_rabbitmq_protobuf(rabbitmq_cluster): + instance.query(''' + CREATE TABLE test.rabbitmq (key UInt64, value String) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'pb', + rabbitmq_format = 'Protobuf', + rabbitmq_schema = 'rabbitmq.proto:KeyValuePair'; + ''') + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + + data = '' + for i in range(0, 20): + msg = rabbitmq_pb2.KeyValuePair() + msg.key = i + msg.value = str(i) + serialized_msg = msg.SerializeToString() + data = data + _VarintBytes(len(serialized_msg)) + serialized_msg + channel.basic_publish(exchange='pb', routing_key='', body=data) + data = '' + for i in range(20, 21): + msg = rabbitmq_pb2.KeyValuePair() + msg.key = i + msg.value = str(i) + serialized_msg = msg.SerializeToString() + data = data + _VarintBytes(len(serialized_msg)) + serialized_msg + channel.basic_publish(exchange='pb', routing_key='', body=data) + data = '' + for i in range(21, 50): + msg = rabbitmq_pb2.KeyValuePair() + msg.key = i + msg.value = str(i) + serialized_msg = msg.SerializeToString() + data = data + _VarintBytes(len(serialized_msg)) + serialized_msg + channel.basic_publish(exchange='pb', routing_key='', body=data) + + result = '' + while True: + result += instance.query('SELECT * FROM test.rabbitmq') + if rabbitmq_check_result(result): + break + + connection.close() + rabbitmq_check_result(result, True) + + @pytest.mark.timeout(180) def test_rabbitmq_materialized_view(rabbitmq_cluster): instance.query(''' @@ -451,6 +508,7 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): DROP TABLE test.view2; ''') + connection.close() rabbitmq_check_result(result1, True) rabbitmq_check_result(result2, True) @@ -1440,7 +1498,7 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): connection.close() result = instance.query(''' - SELECT key, value, _exchange_name, SUBSTRING(_channel_id, 34, 3), _delivery_tag, _redelivered + SELECT key, value, _exchange_name, SUBSTRING(_channel_id, 1, 3), _delivery_tag, _redelivered FROM test.view ORDER BY key ''') @@ -1505,7 +1563,7 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): connection.close() - result = instance.query("SELECT key, value, exchange_name, SUBSTRING(channel_id, 34, 3), delivery_tag, redelivered FROM test.view ORDER BY delivery_tag") + result = instance.query("SELECT key, value, exchange_name, SUBSTRING(channel_id, 1, 3), delivery_tag, redelivered FROM test.view ORDER BY delivery_tag") expected = '''\ 0 0 virtuals_mv 1_0 1 0 1 1 virtuals_mv 1_0 2 0 @@ -1769,7 +1827,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): @pytest.mark.timeout(420) -def test_rabbitmq_consumer_restore_failed_connection_without_losses(rabbitmq_cluster): +def test_rabbitmq_consumer_restore_failed_connection_without_losses_1(rabbitmq_cluster): instance.query(''' CREATE TABLE test.consumer_reconnect (key UInt64, value UInt64) ENGINE = RabbitMQ @@ -1901,71 +1959,72 @@ def test_rabbitmq_producer_restore_failed_connection_without_losses(rabbitmq_clu @pytest.mark.timeout(420) -def test_rabbitmq_virtual_columns_2(rabbitmq_cluster): +def test_rabbitmq_consumer_restore_failed_connection_without_losses_2(rabbitmq_cluster): instance.query(''' - DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64, - exchange_name String, channel_id String, delivery_tag UInt64, redelivered UInt8) ENGINE = MergeTree() - ORDER BY key; + CREATE TABLE test.consumer_reconnect (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'consumer_reconnect', + rabbitmq_num_consumers = 10, + rabbitmq_num_queues = 2, + rabbitmq_format = 'JSONEachRow', + rabbitmq_row_delimiter = '\\n'; ''') - table_num = 3 - for table_id in range(table_num): - print("Setting up table {}".format(table_id)) - instance.query(''' - DROP TABLE IF EXISTS test.virtuals_{0}; - DROP TABLE IF EXISTS test.virtuals_{0}_mv; - CREATE TABLE test.virtuals_{0} (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'virtuals_2', - rabbitmq_num_queues = 2, - rabbitmq_num_consumers = 2, - rabbitmq_format = 'JSONEachRow', - rabbitmq_row_delimiter = '\\n'; - CREATE MATERIALIZED VIEW test.many_consumers_{0}_mv TO test.destination AS - SELECT *, _exchange_name as exchange_name, _channel_id as channel_id, _delivery_tag as delivery_tag, _redelivered as redelivered - FROM test.virtuals_{0}; - '''.format(table_id)) + i = 0 + messages_num = 150000 credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + connection = pika.BlockingConnection(parameters) channel = connection.channel() - - message_num = 10 - i = 0 messages = [] - for _ in range(message_num): + for _ in range(messages_num): messages.append(json.dumps({'key': i, 'value': i})) i += 1 + for i in range(messages_num): + channel.basic_publish(exchange='consumer_reconnect', routing_key='', body=messages[i], + properties=pika.BasicProperties(delivery_mode = 2, message_id=str(i))) + connection.close() - for i in range(message_num): - channel.basic_publish(exchange='virtuals_2', routing_key='', body=messages[i], - properties=pika.BasicProperties(delivery_mode=2, message_id=str(i))) + instance.query(''' + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.consumer_reconnect; + ''') - #kill_rabbitmq(); - #time.sleep(2); - #revive_rabbitmq(); + while int(instance.query('SELECT count() FROM test.view')) == 0: + time.sleep(0.1) + + kill_rabbitmq(); + time.sleep(8); + revive_rabbitmq(); + + while int(instance.query('SELECT count() FROM test.view')) == 0: + time.sleep(0.1) + + kill_rabbitmq(); + time.sleep(2); + revive_rabbitmq(); while True: - result = instance.query('SELECT count(DISTINCT concat([channel_id], [toString(delivery_tag)])) FROM test.destination') - print instance.query(''' - SELECT DISTINCT concat([channel_id], [toString(delivery_tag)]) - FROM (SELECT channel_id AS id, delivery_tag AS tag FROM test.destination GROUP BY id ORDER BY tag)''') + result = instance.query('SELECT count(DISTINCT key) FROM test.view') time.sleep(1) - if int(result) == message_num * table_num: + if int(result) == messages_num: break - connection.close() - instance.query(''' DROP TABLE IF EXISTS test.consumer; DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.rabbitmq_virtuals_mv + DROP TABLE IF EXISTS test.consumer_reconnect; ''') - assert int(result) == message_num * table_num + assert int(result) == messages_num, 'ClickHouse lost some messages: {}'.format(result) if __name__ == '__main__': From fb1417db7188a5b83c8a02344993597e054c7db1 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 16 Aug 2020 01:08:03 +0800 Subject: [PATCH 0058/1911] ISSUES-4006 try fix test failure --- src/DataTypes/registerDataTypeDateTime.cpp | 41 ++++++++++++------- src/Functions/FunctionsConversion.h | 25 +++++++---- .../0_stateless/00921_datetime64_basic.sql | 4 +- .../01442_date_time_with_params.reference | 6 +-- .../01442_date_time_with_params.sql | 10 ++--- 5 files changed, 53 insertions(+), 33 deletions(-) diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index c6a79e48335..0596b229494 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -22,6 +22,16 @@ enum class ArgumentKind Mandatory }; +String getExceptionMessage( + const String & message, size_t argument_index, const char * argument_name, + const std::string & context_data_type_name, Field::Types::Which field_type) +{ + return std::string("Parameter #") + std::to_string(argument_index) + " '" + + argument_name + "' for " + context_data_type_name + + message + + ", expected: " + Field::Types::toString(field_type) + " literal."; +} + template std::conditional_t, T> getArgument(const ASTPtr & arguments, size_t argument_index, const char * argument_name, const std::string context_data_type_name) @@ -30,14 +40,6 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume const auto field_type = Field::TypeToEnum::value; const ASTLiteral * argument = nullptr; - auto exception_message = [=](const String & message) - { - return std::string("Parameter #") + std::to_string(argument_index) + " '" - + argument_name + "' for " + context_data_type_name - + message - + ", expected: " + Field::Types::toString(field_type) + " literal."; - }; - if (!arguments || arguments->children.size() <= argument_index || !(argument = arguments->children[argument_index]->as()) || argument->value.getType() != field_type) @@ -45,8 +47,8 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume if constexpr (Kind == ArgumentKind::Optional) return {}; else - throw Exception(exception_message(" is missing"), - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(getExceptionMessage(" is missing", argument_index, argument_name, context_data_type_name, field_type), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } return argument->value.get(); @@ -54,21 +56,26 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume static DataTypePtr create(const ASTPtr & arguments) { - if (!arguments || arguments->size() == 0) + if (!arguments || arguments->children.size() == 0) return std::make_shared(); const auto scale = getArgument(arguments, 0, "scale", "DateTime"); const auto timezone = getArgument(arguments, !!scale, "timezone", "DateTime"); - if (scale) - return std::make_shared(scale.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{})); + if (!scale && !timezone) + throw Exception(getExceptionMessage(" has wrong type: ", 0, "scale", "DateTime", Field::Types::Which::UInt64), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + /// If scale is defined, the data type is DateTime when scale = 0 otherwise the data type is DateTime64 + if (scale && scale.value() != 0) + return std::make_shared(scale.value(), timezone.value_or(String{})); return std::make_shared(timezone.value_or(String{})); } static DataTypePtr create32(const ASTPtr & arguments) { - if (!arguments || arguments->size() == 0) + if (!arguments || arguments->children.size() == 0) return std::make_shared(); if (arguments->children.size() != 1) @@ -81,7 +88,7 @@ static DataTypePtr create32(const ASTPtr & arguments) static DataTypePtr create64(const ASTPtr & arguments) { - if (!arguments || arguments->size() == 0) + if (!arguments || arguments->children.size() == 0) return std::make_shared(DataTypeDateTime64::default_scale); if (arguments->children.size() > 2) @@ -90,6 +97,10 @@ static DataTypePtr create64(const ASTPtr & arguments) const auto scale = getArgument(arguments, 0, "scale", "DateTime64"); const auto timezone = getArgument(arguments, !!scale, "timezone", "DateTime64"); + if (!scale && !timezone) + throw Exception(getExceptionMessage(" has wrong type: ", 0, "scale", "DateTime", Field::Types::Which::UInt64), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + return std::make_shared(scale.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{})); } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index a8e8ad81ff8..9e5a781240d 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1029,7 +1029,7 @@ public: mandatory_args.push_back({"scale", &isNativeInteger, &isColumnConst, "const Integer"}); } - if constexpr (std::is_same_v && std::is_same_v) + if constexpr (std::is_same_v) { /// toDateTime(value, scale:Integer) if ((arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3) @@ -1085,14 +1085,16 @@ public: scale = static_cast(arguments[1].column->get64(0)); } - if constexpr (std::is_same_v && std::is_same_v) + if constexpr (std::is_same_v) { /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 if ((arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3) { timezone_arg_position += 1; scale = static_cast(arguments[1].column->get64(0)); - return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); + if (scale != 0) /// toDateTime('xxxx-xx-xx xx:xx:xx', 0) return DateTime + return std::make_shared( + scale, extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); } } @@ -1199,15 +1201,22 @@ private: return true; }; - if constexpr (std::is_same_v && std::is_same_v) + if constexpr (std::is_same_v) { /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 if ((arguments.size() == 2 && isUnsignedInteger(block.getByPosition(arguments[1]).type)) || arguments.size() == 3) { - if (!callOnIndexAndDataType(from_type->getTypeId(), call)) - throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return; + const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]); + UInt32 scale = extractToDecimalScale(scale_column); + + if (scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64 + { + if (!callOnIndexAndDataType(from_type->getTypeId(), call)) + throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return; + } } } diff --git a/tests/queries/0_stateless/00921_datetime64_basic.sql b/tests/queries/0_stateless/00921_datetime64_basic.sql index 2d7cb975cfc..bc881e3175d 100644 --- a/tests/queries/0_stateless/00921_datetime64_basic.sql +++ b/tests/queries/0_stateless/00921_datetime64_basic.sql @@ -1,11 +1,11 @@ DROP TABLE IF EXISTS A; -SELECT CAST(1 as DateTime64('abc')); -- { serverError 43 } # Invalid scale parameter type +SELECT CAST(1 as DateTime64('abc')); -- { serverError 1000 } # invalid timezone SELECT CAST(1 as DateTime64(100)); -- { serverError 69 } # too big scale SELECT CAST(1 as DateTime64(-1)); -- { serverError 43 } # signed scale parameter type SELECT CAST(1 as DateTime64(3, 'qqq')); -- { serverError 1000 } # invalid timezone -SELECT toDateTime64('2019-09-16 19:20:11.234', 'abc'); -- { serverError 43 } # invalid scale +SELECT toDateTime64('2019-09-16 19:20:11.234', 'abc'); -- { serverError 43 } # invalid timezone SELECT toDateTime64('2019-09-16 19:20:11.234', 100); -- { serverError 69 } # too big scale SELECT toDateTime64(CAST([['CLb5Ph ']], 'String'), uniqHLL12('2Gs1V', 752)); -- { serverError 44 } # non-const string and non-const scale SELECT toDateTime64('2019-09-16 19:20:11.234', 3, 'qqq'); -- { serverError 1000 } # invalid timezone diff --git a/tests/queries/0_stateless/01442_date_time_with_params.reference b/tests/queries/0_stateless/01442_date_time_with_params.reference index a6cb7f7b948..03b591a34bb 100644 --- a/tests/queries/0_stateless/01442_date_time_with_params.reference +++ b/tests/queries/0_stateless/01442_date_time_with_params.reference @@ -1,4 +1,4 @@ -2020-01-01 00:00:00 DateTime 2020-01-01 00:01:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') 2020-01-01 00:05:00 DateTime 2020-01-01 00:06:00 DateTime(\'Europe/Moscow\') -2020-01-01 00:00:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') -2020-01-01 00:00:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') +2020-01-01 00:00:00 DateTime 2020-01-01 00:01:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') 2020-01-01 00:05:00 DateTime 2020-01-01 00:06:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:06:00 DateTime +2020-01-01 00:00:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') 2020-01-01 00:05:00 DateTime +2020-01-01 00:00:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') 2020-01-01 00:05:00 DateTime 2020-01-01 00:00:00 DateTime diff --git a/tests/queries/0_stateless/01442_date_time_with_params.sql b/tests/queries/0_stateless/01442_date_time_with_params.sql index 1e75089bc05..d2664a4e316 100644 --- a/tests/queries/0_stateless/01442_date_time_with_params.sql +++ b/tests/queries/0_stateless/01442_date_time_with_params.sql @@ -1,14 +1,14 @@ DROP TABLE IF EXISTS test; -CREATE TABLE test (a DateTime, b DateTime(), c DateTime(2), d DateTime('Europe/Moscow'), e DateTime(3, 'Europe/Moscow'), f DateTime32, g DateTime32('Europe/Moscow')) ENGINE = MergeTree ORDER BY a; +CREATE TABLE test (a DateTime, b DateTime(), c DateTime(2), d DateTime('Europe/Moscow'), e DateTime(3, 'Europe/Moscow'), f DateTime32, g DateTime32('Europe/Moscow'), h DateTime(0)) ENGINE = MergeTree ORDER BY a; -INSERT INTO test VALUES('2020-01-01 00:00:00', '2020-01-01 00:01:00', '2020-01-01 00:02:00.11', '2020-01-01 00:03:00', '2020-01-01 00:04:00.22', '2020-01-01 00:05:00', '2020-01-01 00:06:00') +INSERT INTO test VALUES('2020-01-01 00:00:00', '2020-01-01 00:01:00', '2020-01-01 00:02:00.11', '2020-01-01 00:03:00', '2020-01-01 00:04:00.22', '2020-01-01 00:05:00', '2020-01-01 00:06:00', '2020-01-01 00:06:00'); -SELECT a, toTypeName(a), b, toTypeName(b), c, toTypeName(c), d, toTypeName(d), e, toTypeName(e), f, toTypeName(f), g, toTypeName(g) FROM test; +SELECT a, toTypeName(a), b, toTypeName(b), c, toTypeName(c), d, toTypeName(d), e, toTypeName(e), f, toTypeName(f), g, toTypeName(g), h, toTypeName(h) FROM test; -SELECT toDateTime('2020-01-01 00:00:00') AS a, toTypeName(a), toDateTime('2020-01-01 00:02:00.11', 2) AS b, toTypeName(b), toDateTime('2020-01-01 00:03:00', 'Europe/Moscow') AS c, toTypeName(c), toDateTime('2020-01-01 00:04:00.22', 3, 'Europe/Moscow') AS d, toTypeName(d); +SELECT toDateTime('2020-01-01 00:00:00') AS a, toTypeName(a), toDateTime('2020-01-01 00:02:00.11', 2) AS b, toTypeName(b), toDateTime('2020-01-01 00:03:00', 'Europe/Moscow') AS c, toTypeName(c), toDateTime('2020-01-01 00:04:00.22', 3, 'Europe/Moscow') AS d, toTypeName(d), toDateTime('2020-01-01 00:05:00', 0) AS e, toTypeName(e); -SELECT CAST('2020-01-01 00:00:00', 'DateTime') AS a, toTypeName(a), CAST('2020-01-01 00:02:00.11', 'DateTime(2)') AS b, toTypeName(b), CAST('2020-01-01 00:03:00', 'DateTime(\'Europe/Moscow\')') AS c, toTypeName(c), CAST('2020-01-01 00:04:00.22', 'DateTime(3, \'Europe/Moscow\')') AS d, toTypeName(d); +SELECT CAST('2020-01-01 00:00:00', 'DateTime') AS a, toTypeName(a), CAST('2020-01-01 00:02:00.11', 'DateTime(2)') AS b, toTypeName(b), CAST('2020-01-01 00:03:00', 'DateTime(\'Europe/Moscow\')') AS c, toTypeName(c), CAST('2020-01-01 00:04:00.22', 'DateTime(3, \'Europe/Moscow\')') AS d, toTypeName(d), CAST('2020-01-01 00:05:00', 'DateTime(0)') AS e, toTypeName(e); SELECT toDateTime32('2020-01-01 00:00:00') AS a, toTypeName(a); From ade8c19b571f1f0ab1eb47727bd48341c1219f6d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 16 Aug 2020 13:21:38 +0800 Subject: [PATCH 0059/1911] ISSUES-4006 try fix build & test failure --- src/DataTypes/registerDataTypeDateTime.cpp | 16 ++++++---------- .../0_stateless/00921_datetime64_basic.sql | 6 +++--- 2 files changed, 9 insertions(+), 13 deletions(-) diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index 0596b229494..9b6af5f6e0b 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -34,7 +34,7 @@ String getExceptionMessage( template std::conditional_t, T> -getArgument(const ASTPtr & arguments, size_t argument_index, const char * argument_name, const std::string context_data_type_name) +getArgument(const ASTPtr & arguments, size_t argument_index, const char * argument_name [[maybe_unused]], const std::string context_data_type_name) { using NearestResultType = NearestFieldType; const auto field_type = Field::TypeToEnum::value; @@ -56,7 +56,7 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume static DataTypePtr create(const ASTPtr & arguments) { - if (!arguments || arguments->children.size() == 0) + if (!arguments || arguments->children.empty()) return std::make_shared(); const auto scale = getArgument(arguments, 0, "scale", "DateTime"); @@ -75,7 +75,7 @@ static DataTypePtr create(const ASTPtr & arguments) static DataTypePtr create32(const ASTPtr & arguments) { - if (!arguments || arguments->children.size() == 0) + if (!arguments || arguments->children.empty()) return std::make_shared(); if (arguments->children.size() != 1) @@ -88,20 +88,16 @@ static DataTypePtr create32(const ASTPtr & arguments) static DataTypePtr create64(const ASTPtr & arguments) { - if (!arguments || arguments->children.size() == 0) + if (!arguments || arguments->children.empty()) return std::make_shared(DataTypeDateTime64::default_scale); if (arguments->children.size() > 2) throw Exception("DateTime64 data type can optionally have two argument - scale and time zone name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - const auto scale = getArgument(arguments, 0, "scale", "DateTime64"); + const auto scale = getArgument(arguments, 0, "scale", "DateTime64"); const auto timezone = getArgument(arguments, !!scale, "timezone", "DateTime64"); - if (!scale && !timezone) - throw Exception(getExceptionMessage(" has wrong type: ", 0, "scale", "DateTime", Field::Types::Which::UInt64), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(scale.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{})); + return std::make_shared(scale, timezone.value_or(String{})); } void registerDataTypeDateTime(DataTypeFactory & factory) diff --git a/tests/queries/0_stateless/00921_datetime64_basic.sql b/tests/queries/0_stateless/00921_datetime64_basic.sql index bc881e3175d..1fc534d8afd 100644 --- a/tests/queries/0_stateless/00921_datetime64_basic.sql +++ b/tests/queries/0_stateless/00921_datetime64_basic.sql @@ -1,11 +1,11 @@ DROP TABLE IF EXISTS A; -SELECT CAST(1 as DateTime64('abc')); -- { serverError 1000 } # invalid timezone +SELECT CAST(1 as DateTime64('abc')); -- { serverError 42 } # Miss scale parameter type SELECT CAST(1 as DateTime64(100)); -- { serverError 69 } # too big scale -SELECT CAST(1 as DateTime64(-1)); -- { serverError 43 } # signed scale parameter type +SELECT CAST(1 as DateTime64(-1)); -- { serverError 42 } # Miss scale parameter type SELECT CAST(1 as DateTime64(3, 'qqq')); -- { serverError 1000 } # invalid timezone -SELECT toDateTime64('2019-09-16 19:20:11.234', 'abc'); -- { serverError 43 } # invalid timezone +SELECT toDateTime64('2019-09-16 19:20:11.234', 'abc'); -- { serverError 43 } # Miss scale parameter type SELECT toDateTime64('2019-09-16 19:20:11.234', 100); -- { serverError 69 } # too big scale SELECT toDateTime64(CAST([['CLb5Ph ']], 'String'), uniqHLL12('2Gs1V', 752)); -- { serverError 44 } # non-const string and non-const scale SELECT toDateTime64('2019-09-16 19:20:11.234', 3, 'qqq'); -- { serverError 1000 } # invalid timezone From 405a6fb08fa22a9e063dd5e48e7ee6060f718749 Mon Sep 17 00:00:00 2001 From: hexiaoting <“hewenting_ict@163.com”> Date: Mon, 17 Aug 2020 18:20:23 +0800 Subject: [PATCH 0060/1911] New feature: LineAsString format. #13630 --- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatFactory.h | 1 + .../Impl/LineAsStringRowInputFormat.cpp | 101 ++++++++++++++++++ .../Formats/Impl/LineAsStringRowInputFormat.h | 31 ++++++ src/Processors/ya.make | 1 + 5 files changed, 135 insertions(+) create mode 100644 src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp create mode 100644 src/Processors/Formats/Impl/LineAsStringRowInputFormat.h diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 5256ab2b321..f996e3d8cf2 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -365,6 +365,7 @@ FormatFactory::FormatFactory() registerInputFormatProcessorMsgPack(*this); registerOutputFormatProcessorMsgPack(*this); registerInputFormatProcessorJSONAsString(*this); + registerInputFormatProcessorLineAsString(*this); registerFileSegmentationEngineTabSeparated(*this); registerFileSegmentationEngineCSV(*this); diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index ea4004c191f..610cf8105b8 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -210,5 +210,6 @@ void registerOutputFormatProcessorPostgreSQLWire(FormatFactory & factory); void registerInputFormatProcessorCapnProto(FormatFactory & factory); void registerInputFormatProcessorRegexp(FormatFactory & factory); void registerInputFormatProcessorJSONAsString(FormatFactory & factory); +void registerInputFormatProcessorLineAsString(FormatFactory & factory); } diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp new file mode 100644 index 00000000000..a28b3903724 --- /dev/null +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -0,0 +1,101 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int INCORRECT_DATA; +} + +LineAsStringRowInputFormat::LineAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) : + IRowInputFormat(header_, in_, std::move(params_)), buf(in) +{ + if (header_.columns() > 1 || header_.getDataTypes()[0]->getTypeId() != TypeIndex::String) + { + throw Exception("This input format is only suitable for tables with a single column of type String.", ErrorCodes::LOGICAL_ERROR); + } +} + +void LineAsStringRowInputFormat::resetParser() +{ + IRowInputFormat::resetParser(); + buf.reset(); +} + +void LineAsStringRowInputFormat::readLineObject(IColumn & column) +{ + PeekableReadBufferCheckpoint checkpoint{buf}; + size_t balance = 0; + + if (*buf.position() != '"') + throw Exception("Line object must begin with '\"'.", ErrorCodes::INCORRECT_DATA); + + ++buf.position(); + ++balance; + + char * pos; + + while (balance) + { + if (buf.eof()) + throw Exception("Unexpected end of file while parsing Line object.", ErrorCodes::INCORRECT_DATA); + + pos = find_last_symbols_or_null<'"', '\\'>(buf.position(), buf.buffer().end()); + buf.position() = pos; + if (buf.position() == buf.buffer().end()) + continue; + else if (*buf.position() == '"') + { + --balance; + ++buf.position(); + } + else if (*buf.position() == '\\') + { + ++buf.position(); + if (!buf.eof()) + { + ++buf.position(); + } + } + + } + buf.makeContinuousMemoryFromCheckpointToPos(); + char * end = buf.position(); + buf.rollbackToCheckpoint(); + column.insertData(buf.position(), end - buf.position()); + buf.position() = end; +} + +bool LineAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) +{ + skipWhitespaceIfAny(buf); + + if (!buf.eof()) + readLineObject(*columns[0]); + + skipWhitespaceIfAny(buf); + if (!buf.eof() && *buf.position() == ',') + ++buf.position(); + skipWhitespaceIfAny(buf); + + return !buf.eof(); +} + +void registerInputFormatProcessorLineAsString(FormatFactory & factory) +{ + factory.registerInputFormatProcessor("LineAsString", []( + ReadBuffer & buf, + const Block & sample, + const RowInputFormatParams & params, + const FormatSettings &) + { + return std::make_shared(sample, buf, params); + }); +} + +} diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h new file mode 100644 index 00000000000..a31dce1cc4a --- /dev/null +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class ReadBuffer; + +/// This format parses a sequence of Line objects separated by newlines, spaces and/or comma. +/// Each Line object is parsed as a whole to string. +/// This format can only parse a table with single field of type String. + +class LineAsStringRowInputFormat : public IRowInputFormat +{ +public: + LineAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_); + + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + String getName() const override { return "LineAsStringRowInputFormat"; } + void resetParser() override; + +private: + void readLineObject(IColumn & column); + + PeekableReadBuffer buf; +}; + +} diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 4c25ad5bf3f..081b1d5ba1f 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -23,6 +23,7 @@ SRCS( Formats/Impl/ConstantExpressionTemplate.cpp Formats/Impl/CSVRowInputFormat.cpp Formats/Impl/CSVRowOutputFormat.cpp + Formats/Impl/LineAsStringRowInputFormat.cpp Formats/Impl/JSONAsStringRowInputFormat.cpp Formats/Impl/JSONCompactEachRowRowInputFormat.cpp Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp From 0451d5729323b7f46d79336fea4f0982bb1662ae Mon Sep 17 00:00:00 2001 From: hexiaoting <“hewenting_ict@163.com”> Date: Tue, 18 Aug 2020 10:35:08 +0800 Subject: [PATCH 0061/1911] Add new feature: LineAsString Format --- src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp | 6 ++++++ .../0_stateless/01460_line_as_string_format.reference | 1 + tests/queries/0_stateless/01460_line_as_string_format.sql | 5 +++++ 3 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/01460_line_as_string_format.reference create mode 100644 tests/queries/0_stateless/01460_line_as_string_format.sql diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp index a28b3903724..36844fa700b 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -32,6 +32,12 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column) PeekableReadBufferCheckpoint checkpoint{buf}; size_t balance = 0; + if (*buf.position() == ';') { + ++buf.position(); + if(buf.eof()) + return; + } + if (*buf.position() != '"') throw Exception("Line object must begin with '\"'.", ErrorCodes::INCORRECT_DATA); diff --git a/tests/queries/0_stateless/01460_line_as_string_format.reference b/tests/queries/0_stateless/01460_line_as_string_format.reference new file mode 100644 index 00000000000..989f8ac0292 --- /dev/null +++ b/tests/queries/0_stateless/01460_line_as_string_format.reference @@ -0,0 +1 @@ +"I love apple","I love banana","I love pear" diff --git a/tests/queries/0_stateless/01460_line_as_string_format.sql b/tests/queries/0_stateless/01460_line_as_string_format.sql new file mode 100644 index 00000000000..e5518a828d0 --- /dev/null +++ b/tests/queries/0_stateless/01460_line_as_string_format.sql @@ -0,0 +1,5 @@ +DROP TABLE IF EXISTS line_as_string; +CREATE TABLE line_as_string (field String) ENGINE = Memory; +INSERT INTO line_as_string FORMAT LineAsString "I love apple","I love banana","I love pear"; +SELECT * FROM line_as_string; +DROP TABLE line_as_string; From ccbfd2e5df5d9cfd6c88c7717cf53344bee8bf3f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 18 Aug 2020 17:16:40 +0300 Subject: [PATCH 0062/1911] Update deb-builder ubuntu version. Add clang-11, gcc-10. Add unbundled docker. Add test-base image. --- docker/images.json | 5 +++ docker/packager/deb/Dockerfile | 49 +++++------------------- docker/packager/packager | 7 +++- docker/packager/unbundled/Dockerfile | 56 ++++++++++++++++++++++++++++ docker/packager/unbundled/build.sh | 18 +++++++++ docker/test/base/Dockerfile | 51 +++++++++++++++++++++++++ 6 files changed, 144 insertions(+), 42 deletions(-) create mode 100644 docker/packager/unbundled/Dockerfile create mode 100755 docker/packager/unbundled/build.sh create mode 100644 docker/test/base/Dockerfile diff --git a/docker/images.json b/docker/images.json index 0ab1688efb0..e9e83f7859e 100644 --- a/docker/images.json +++ b/docker/images.json @@ -2,6 +2,7 @@ "docker/packager/deb": { "name": "yandex/clickhouse-deb-builder", "dependent": [ + "docker/packager/unbundled", "docker/test/stateless", "docker/test/stateless_with_coverage", "docker/test/stateless_pytest", @@ -15,6 +16,10 @@ "docker/test/pvs" ] }, + "docker/packager/unbundled": { + "name": "yandex/clickhouse-unbundled-builder", + "dependent": [] + }, "docker/test/coverage": { "name": "yandex/clickhouse-coverage", "dependent": [] diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index c1260b5c7ff..b1f711509fb 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -1,9 +1,9 @@ # docker build -t yandex/clickhouse-deb-builder . -FROM ubuntu:19.10 +FROM ubuntu:20.04 RUN apt-get --allow-unauthenticated update -y && apt-get install --yes wget gnupg RUN wget -O - https://apt.llvm.org/llvm-snapshot.gpg.key | apt-key add - -RUN echo "deb [trusted=yes] http://apt.llvm.org/eoan/ llvm-toolchain-eoan-10 main" >> /etc/apt/sources.list +RUN echo "deb [trusted=yes] http://apt.llvm.org/focal/ llvm-toolchain-focal-11 main" >> /etc/apt/sources.list # initial packages RUN apt-get --allow-unauthenticated update -y \ @@ -25,13 +25,17 @@ RUN curl -O https://clickhouse-builds.s3.yandex.net/utils/1/dpkg-deb RUN chmod +x dpkg-deb RUN cp dpkg-deb /usr/bin - -# Libraries from OS are only needed to test the "unbundled" build (that is not used in production). RUN apt-get --allow-unauthenticated update -y \ && env DEBIAN_FRONTEND=noninteractive \ apt-get --allow-unauthenticated install --yes --no-install-recommends \ + gcc-10 \ + g++-10 \ gcc-9 \ g++-9 \ + llvm-11 \ + clang-11 \ + lld-11 \ + clang-tidy-11 \ llvm-10 \ clang-10 \ lld-10 \ @@ -39,54 +43,19 @@ RUN apt-get --allow-unauthenticated update -y \ clang-9 \ lld-9 \ clang-tidy-9 \ - libicu-dev \ - libreadline-dev \ - gperf \ ninja-build \ perl \ pkg-config \ devscripts \ debhelper \ git \ - libc++-dev \ - libc++abi-dev \ - libboost-program-options-dev \ - libboost-system-dev \ - libboost-filesystem-dev \ - libboost-thread-dev \ - libboost-iostreams-dev \ - libboost-regex-dev \ - zlib1g-dev \ - liblz4-dev \ - libdouble-conversion-dev \ - librdkafka-dev \ - libpoconetssl62 \ - libpoco-dev \ - libgoogle-perftools-dev \ - libzstd-dev \ - libltdl-dev \ - libre2-dev \ - libjemalloc-dev \ - libmsgpack-dev \ - libcurl4-openssl-dev \ - opencl-headers \ - ocl-icd-libopencl1 \ - intel-opencl-icd \ - unixodbc-dev \ - odbcinst \ tzdata \ gperf \ alien \ - libcapnp-dev \ cmake \ gdb \ - pigz \ moreutils \ - libcctz-dev \ - libldap2-dev \ - libsasl2-dev \ - heimdal-multidev \ - libhyperscan-dev + pigz # This symlink required by gcc to find lld compiler diff --git a/docker/packager/packager b/docker/packager/packager index bc97429336c..63652da2439 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -11,6 +11,7 @@ SCRIPT_PATH = os.path.realpath(__file__) IMAGE_MAP = { "deb": "yandex/clickhouse-deb-builder", "binary": "yandex/clickhouse-binary-builder", + "unbundled": "yandex/clickhouse-unbundled-builder" } def check_image_exists_locally(image_name): @@ -176,7 +177,9 @@ if __name__ == "__main__": parser.add_argument("--clickhouse-repo-path", default=os.path.join(os.path.dirname(os.path.abspath(__file__)), os.pardir, os.pardir)) parser.add_argument("--output-dir", required=True) parser.add_argument("--build-type", choices=("debug", ""), default="") - parser.add_argument("--compiler", choices=("clang-10-darwin", "clang-10-aarch64", "clang-10-freebsd", "gcc-9", "clang-10"), default="gcc-9") + parser.add_argument("--compiler", choices=("clang-10", "clang-10-darwin", "clang-10-aarch64", "clang-10-freebsd", + "clang-11", "clang-11-darwin", "clang-11-aarch64", "clang-11-freebsd", + "gcc-9", "gcc-10"), default="gcc-9") parser.add_argument("--sanitizer", choices=("address", "thread", "memory", "undefined", ""), default="") parser.add_argument("--unbundled", action="store_true") parser.add_argument("--split-binary", action="store_true") @@ -197,7 +200,7 @@ if __name__ == "__main__": if not os.path.isabs(args.output_dir): args.output_dir = os.path.abspath(os.path.join(os.getcwd(), args.output_dir)) - image_type = 'binary' if args.package_type == 'performance' else args.package_type + image_type = 'binary' if args.package_type == 'performance' else 'unbundled' if args.unbundled else args.package_type image_name = IMAGE_MAP[image_type] if not os.path.isabs(args.clickhouse_repo_path): diff --git a/docker/packager/unbundled/Dockerfile b/docker/packager/unbundled/Dockerfile new file mode 100644 index 00000000000..d099be6cb90 --- /dev/null +++ b/docker/packager/unbundled/Dockerfile @@ -0,0 +1,56 @@ +# docker build -t yandex/clickhouse-unbundled-builder . +FROM yandex/clickhouse-deb-builder + +# Libraries from OS are only needed to test the "unbundled" build (that is not used in production). +RUN apt-get --allow-unauthenticated update -y \ + && env DEBIAN_FRONTEND=noninteractive \ + apt-get --allow-unauthenticated install --yes --no-install-recommends \ + libicu-dev \ + libreadline-dev \ + gperf \ + perl \ + pkg-config \ + devscripts \ + libc++-dev \ + libc++abi-dev \ + libboost-program-options-dev \ + libboost-system-dev \ + libboost-filesystem-dev \ + libboost-thread-dev \ + libboost-iostreams-dev \ + libboost-regex-dev \ + zlib1g-dev \ + liblz4-dev \ + libdouble-conversion-dev \ + librdkafka-dev \ + libpoconetssl62 \ + libpoco-dev \ + libgoogle-perftools-dev \ + libzstd-dev \ + libltdl-dev \ + libre2-dev \ + libjemalloc-dev \ + libmsgpack-dev \ + libcurl4-openssl-dev \ + opencl-headers \ + ocl-icd-libopencl1 \ + intel-opencl-icd \ + unixodbc-dev \ + odbcinst \ + tzdata \ + gperf \ + alien \ + libcapnp-dev \ + cmake \ + gdb \ + pigz \ + moreutils \ + libcctz-dev \ + libldap2-dev \ + libsasl2-dev \ + heimdal-multidev \ + libhyperscan-dev + +COPY build.sh / + +CMD ["/bin/bash", "/build.sh"] diff --git a/docker/packager/unbundled/build.sh b/docker/packager/unbundled/build.sh new file mode 100755 index 00000000000..9fd246bcc33 --- /dev/null +++ b/docker/packager/unbundled/build.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash + +set -x -e + +# Update tzdata to the latest version. It is embedded into clickhouse binary. +sudo apt-get update && sudo apt-get install tzdata + +ccache --show-stats ||: +ccache --zero-stats ||: +build/release --no-pbuilder $ALIEN_PKGS | ts '%Y-%m-%d %H:%M:%S' +mv /*.deb /output +mv *.changes /output +mv *.buildinfo /output +mv /*.rpm /output ||: # if exists +mv /*.tgz /output ||: # if exists + +ccache --show-stats ||: +ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so ||: diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile new file mode 100644 index 00000000000..851ec40a038 --- /dev/null +++ b/docker/test/base/Dockerfile @@ -0,0 +1,51 @@ +# docker build -t yandex/clickhouse-test-base . +FROM ubuntu:19.10 + +RUN apt-get --allow-unauthenticated update -y && apt-get install --yes wget gnupg +RUN wget -O - https://apt.llvm.org/llvm-snapshot.gpg.key | apt-key add - +RUN echo "deb [trusted=yes] http://apt.llvm.org/eoan/ llvm-toolchain-eoan-10 main" >> /etc/apt/sources.list + +# initial packages +RUN apt-get --allow-unauthenticated update -y \ + && env DEBIAN_FRONTEND=noninteractive \ + apt-get --allow-unauthenticated install --yes --no-install-recommends \ + apt-transport-https \ + bash \ + ca-certificates \ + curl \ + fakeroot \ + gnupg \ + software-properties-common + +# Special dpkg-deb (https://github.com/ClickHouse-Extras/dpkg) version which is able +# to compress files using pigz (https://zlib.net/pigz/) instead of gzip. +# Significantly increase deb packaging speed and compatible with old systems +RUN curl -O https://clickhouse-builds.s3.yandex.net/utils/1/dpkg-deb +RUN chmod +x dpkg-deb +RUN cp dpkg-deb /usr/bin + +RUN apt-get --allow-unauthenticated update -y \ + && env DEBIAN_FRONTEND=noninteractive \ + apt-get --allow-unauthenticated install --yes --no-install-recommends \ + clang-10 \ + debhelper \ + devscripts \ + gdb \ + git \ + gperf \ + lcov \ + llvm-10 \ + moreutils \ + perl \ + perl \ + pigz \ + pkg-config \ + tzdata + +# Sanitizer options +RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment; \ + echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ + echo "MSAN_OPTIONS='abort_on_error=1'" >> /etc/environment; \ + ln -s /usr/lib/llvm-10/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; + +CMD sleep 1 \ No newline at end of file From ab3b2c865d46211815a26b1037d87f7282076cb8 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 18 Aug 2020 17:30:21 +0300 Subject: [PATCH 0063/1911] lost chages --- docker/test/coverage/Dockerfile | 25 +----------- docker/test/stateless/Dockerfile | 38 ++++++------------- docker/test/stateless_pytest/Dockerfile | 2 +- .../test/stateless_with_coverage/Dockerfile | 10 +---- 4 files changed, 16 insertions(+), 59 deletions(-) diff --git a/docker/test/coverage/Dockerfile b/docker/test/coverage/Dockerfile index 9ce480f38d8..32020951539 100644 --- a/docker/test/coverage/Dockerfile +++ b/docker/test/coverage/Dockerfile @@ -1,33 +1,12 @@ # docker build -t yandex/clickhouse-coverage . -FROM yandex/clickhouse-deb-builder - -RUN apt-get --allow-unauthenticated update -y \ - && env DEBIAN_FRONTEND=noninteractive \ - apt-get --allow-unauthenticated install --yes --no-install-recommends \ - bash \ - fakeroot \ - cmake \ - ccache \ - curl \ - software-properties-common - - -RUN apt-get --allow-unauthenticated update -y \ - && env DEBIAN_FRONTEND=noninteractive \ - apt-get --allow-unauthenticated install --yes --no-install-recommends \ - perl \ - lcov \ - clang-10 \ - llvm-10 \ - tzdata - +ARG PARENT_TAG=latest +FROM yandex/clickhouse-test-base:${PARENT_TAG} ENV COVERAGE_DIR=/coverage_reports ENV SOURCE_DIR=/build ENV OUTPUT_DIR=/output ENV IGNORE='.*contrib.*' - CMD mkdir -p /build/obj-x86_64-linux-gnu && cd /build/obj-x86_64-linux-gnu && CC=clang-10 CXX=clang++-10 cmake .. && cd /; \ dpkg -i /package_folder/clickhouse-common-static_*.deb; \ llvm-profdata-10 merge -sparse ${COVERAGE_DIR}/* -o clickhouse.profdata && \ diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index f15c2d38971..d3bc03a8f92 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -1,37 +1,29 @@ # docker build -t yandex/clickhouse-stateless-test . -FROM yandex/clickhouse-deb-builder +FROM yandex/clickhouse-test-base ARG odbc_driver_url="https://github.com/ClickHouse/clickhouse-odbc/releases/download/v1.1.4.20200302/clickhouse-odbc-1.1.4-Linux.tar.gz" RUN apt-get update -y \ && env DEBIAN_FRONTEND=noninteractive \ apt-get install --yes --no-install-recommends \ - bash \ - tzdata \ - fakeroot \ - debhelper \ - zookeeper \ - zookeeperd \ + brotli \ expect \ - python \ - python-lxml \ - python-termcolor \ - python-requests \ - curl \ - sudo \ - openssl \ + lsof \ ncdu \ netcat-openbsd \ + openssl \ + python \ + python-lxml \ + python-requests \ + python-termcolor \ + qemu-user-static \ + sudo \ telnet \ tree \ - moreutils \ - brotli \ - gdb \ - lsof \ - llvm-9 \ unixodbc \ wget \ - qemu-user-static + zookeeper \ + zookeeperd RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && wget --quiet -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ @@ -43,11 +35,5 @@ RUN mkdir -p /tmp/clickhouse-odbc-tmp \ ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone -# Sanitizer options -RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment; \ - echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ - echo "MSAN_OPTIONS='abort_on_error=1'" >> /etc/environment; \ - ln -s /usr/lib/llvm-10/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; - COPY run.sh / CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/stateless_pytest/Dockerfile b/docker/test/stateless_pytest/Dockerfile index b3b954a105e..596e2686f49 100644 --- a/docker/test/stateless_pytest/Dockerfile +++ b/docker/test/stateless_pytest/Dockerfile @@ -1,5 +1,5 @@ # docker build -t yandex/clickhouse-stateless-pytest . -FROM yandex/clickhouse-deb-builder +FROM yandex/clickhouse-test-base RUN apt-get update -y && \ apt-get install -y --no-install-recommends \ diff --git a/docker/test/stateless_with_coverage/Dockerfile b/docker/test/stateless_with_coverage/Dockerfile index 9a35f4f4b22..f3539804852 100644 --- a/docker/test/stateless_with_coverage/Dockerfile +++ b/docker/test/stateless_with_coverage/Dockerfile @@ -1,6 +1,6 @@ # docker build -t yandex/clickhouse-stateless-with-coverage-test . # TODO: that can be based on yandex/clickhouse-stateless-test (llvm version and CMD differs) -FROM yandex/clickhouse-deb-builder +FROM yandex/clickhouse-test-base ARG odbc_driver_url="https://github.com/ClickHouse/clickhouse-odbc/releases/download/v1.1.4.20200302/clickhouse-odbc-1.1.4-Linux.tar.gz" @@ -18,7 +18,6 @@ RUN apt-get update -y \ python-lxml \ python-termcolor \ python-requests \ - curl \ sudo \ openssl \ ncdu \ @@ -44,11 +43,4 @@ ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone COPY run.sh /run.sh -# Sanitizer options -RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment; \ - echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ - echo "MSAN_OPTIONS='abort_on_error=1'" >> /etc/environment; \ - ln -s /usr/lib/llvm-10/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; - - CMD ["/bin/bash", "/run.sh"] From adc2c117c8e6b4384fa134988ba2aff19043dec3 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 18 Aug 2020 17:34:04 +0300 Subject: [PATCH 0064/1911] fixes --- tests/integration/test_alter_codec/test.py | 2 -- tests/integration/test_storage_kafka/test.py | 1 - 2 files changed, 3 deletions(-) diff --git a/tests/integration/test_alter_codec/test.py b/tests/integration/test_alter_codec/test.py index 7e038081110..4d251f60b16 100644 --- a/tests/integration/test_alter_codec/test.py +++ b/tests/integration/test_alter_codec/test.py @@ -6,11 +6,9 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', - config_dir='configs', main_configs=['configs/logs_config.xml']) node2 = cluster.add_instance('node2', - config_dir='configs', main_configs=['configs/logs_config.xml']) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 8961fce173f..6d38a7368ea 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -40,7 +40,6 @@ import kafka_pb2 cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', - config_dir='configs', main_configs=['configs/kafka.xml', 'configs/log_conf.xml', 'configs/kafka_macros.xml' ], with_kafka=True, with_zookeeper=True, From 26020cdf6840961e99ee4784307afef68ecee3e3 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 18 Aug 2020 17:38:16 +0300 Subject: [PATCH 0065/1911] typo --- tests/integration/test_distributed_ddl/cluster.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_distributed_ddl/cluster.py b/tests/integration/test_distributed_ddl/cluster.py index b3a0513b799..b8c9527441e 100644 --- a/tests/integration/test_distributed_ddl/cluster.py +++ b/tests/integration/test_distributed_ddl/cluster.py @@ -17,9 +17,9 @@ class ClickHouseClusterWithDDLHelpers(ClickHouseCluster): def prepare(self, replace_hostnames_with_ips=True): try: - main_configs_files = ["clusters.xml", "zookeeper_session_timeout.xml", "macro.xml"), + main_configs_files = ["clusters.xml", "zookeeper_session_timeout.xml", "macro.xml", "query_log.xml","ddl.xml"] - main_configs = [os.path.join(self.test_config_dir, "config.d", f) for f in main_configs_files)] + main_configs = [os.path.join(self.test_config_dir, "config.d", f) for f in main_configs_files] user_configs = [os.path.join(self.test_config_dir, "users.d", f) for f in ["restricted_user.xml", "query_log.xml"]] if self.test_config_dir == "configs_secure": main_configs += [os.path.join(self.test_config_dir, f) for i in ["server.crt", "server.key", "dhparam.pem", "config.d/ssl_conf.xml"]] From e9be2f14ea8ac45f11c7c65b6c36646b64a5b390 Mon Sep 17 00:00:00 2001 From: hexiaoting <“hewenting_ict@163.com”> Date: Wed, 19 Aug 2020 11:50:43 +0800 Subject: [PATCH 0066/1911] fix implementation for \n separated lines --- .../Impl/LineAsStringRowInputFormat.cpp | 45 ++++++------------- .../01460_line_as_string_format.reference | 7 ++- .../01460_line_as_string_format.sh | 19 ++++++++ .../01460_line_as_string_format.sql | 5 --- 4 files changed, 38 insertions(+), 38 deletions(-) create mode 100755 tests/queries/0_stateless/01460_line_as_string_format.sh delete mode 100644 tests/queries/0_stateless/01460_line_as_string_format.sql diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp index 36844fa700b..27bc71d764d 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -30,35 +30,22 @@ void LineAsStringRowInputFormat::resetParser() void LineAsStringRowInputFormat::readLineObject(IColumn & column) { PeekableReadBufferCheckpoint checkpoint{buf}; - size_t balance = 0; - - if (*buf.position() == ';') { - ++buf.position(); - if(buf.eof()) - return; - } - - if (*buf.position() != '"') - throw Exception("Line object must begin with '\"'.", ErrorCodes::INCORRECT_DATA); - - ++buf.position(); - ++balance; + bool newline = true; + bool over = false; char * pos; - while (balance) + while (newline) { - if (buf.eof()) - throw Exception("Unexpected end of file while parsing Line object.", ErrorCodes::INCORRECT_DATA); - - pos = find_last_symbols_or_null<'"', '\\'>(buf.position(), buf.buffer().end()); + pos = find_first_symbols<'\n', '\\'>(buf.position(), buf.buffer().end()); buf.position() = pos; - if (buf.position() == buf.buffer().end()) - continue; - else if (*buf.position() == '"') + if (buf.position() == buf.buffer().end()) { + over = true; + break; + } + else if (*buf.position() == '\n') { - --balance; - ++buf.position(); + newline = false; } else if (*buf.position() == '\\') { @@ -70,25 +57,19 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column) } } + buf.makeContinuousMemoryFromCheckpointToPos(); - char * end = buf.position(); + char * end = over ? buf.position(): ++buf.position(); buf.rollbackToCheckpoint(); - column.insertData(buf.position(), end - buf.position()); + column.insertData(buf.position(), end - (over ? 0 : 1) - buf.position()); buf.position() = end; } bool LineAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { - skipWhitespaceIfAny(buf); - if (!buf.eof()) readLineObject(*columns[0]); - skipWhitespaceIfAny(buf); - if (!buf.eof() && *buf.position() == ',') - ++buf.position(); - skipWhitespaceIfAny(buf); - return !buf.eof(); } diff --git a/tests/queries/0_stateless/01460_line_as_string_format.reference b/tests/queries/0_stateless/01460_line_as_string_format.reference index 989f8ac0292..dec67eb2e0a 100644 --- a/tests/queries/0_stateless/01460_line_as_string_format.reference +++ b/tests/queries/0_stateless/01460_line_as_string_format.reference @@ -1 +1,6 @@ -"I love apple","I love banana","I love pear" +"id" : 1, +"date" : "01.01.2020", +"string" : "123{{{\\"\\\\", +"array" : [1, 2, 3], + +Finally implement this new feature. diff --git a/tests/queries/0_stateless/01460_line_as_string_format.sh b/tests/queries/0_stateless/01460_line_as_string_format.sh new file mode 100755 index 00000000000..a985bc207a8 --- /dev/null +++ b/tests/queries/0_stateless/01460_line_as_string_format.sh @@ -0,0 +1,19 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS line_as_string"; + +$CLICKHOUSE_CLIENT --query="CREATE TABLE line_as_string(field String) ENGINE = Memory"; + +echo '"id" : 1, +"date" : "01.01.2020", +"string" : "123{{{\"\\", +"array" : [1, 2, 3], + +Finally implement this new feature.' | $CLICKHOUSE_CLIENT --query="INSERT INTO line_as_string FORMAT LineAsString"; + +$CLICKHOUSE_CLIENT --query="SELECT * FROM line_as_string"; +$CLICKHOUSE_CLIENT --query="DROP TABLE line_as_string" + diff --git a/tests/queries/0_stateless/01460_line_as_string_format.sql b/tests/queries/0_stateless/01460_line_as_string_format.sql deleted file mode 100644 index e5518a828d0..00000000000 --- a/tests/queries/0_stateless/01460_line_as_string_format.sql +++ /dev/null @@ -1,5 +0,0 @@ -DROP TABLE IF EXISTS line_as_string; -CREATE TABLE line_as_string (field String) ENGINE = Memory; -INSERT INTO line_as_string FORMAT LineAsString "I love apple","I love banana","I love pear"; -SELECT * FROM line_as_string; -DROP TABLE line_as_string; From bdb20738e57f24c84384f78336772cb9efe69ad9 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 19 Aug 2020 13:19:36 +0800 Subject: [PATCH 0067/1911] ISSUES-4006 compatible DateTime64 --- src/DataTypes/registerDataTypeDateTime.cpp | 10 ++++++++-- tests/queries/0_stateless/00921_datetime64_basic.sql | 6 +++--- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index 9b6af5f6e0b..eceb531b892 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -47,8 +47,14 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume if constexpr (Kind == ArgumentKind::Optional) return {}; else - throw Exception(getExceptionMessage(" is missing", argument_index, argument_name, context_data_type_name, field_type), - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + { + if (argument->value.getType() != field_type) + throw Exception(getExceptionMessage(String(" has wrong type: ") + argument->value.getTypeName(), + argument_index, argument_name, context_data_type_name, field_type), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + else + throw Exception(getExceptionMessage(" is missing", argument_index, argument_name, context_data_type_name, field_type), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } } return argument->value.get(); diff --git a/tests/queries/0_stateless/00921_datetime64_basic.sql b/tests/queries/0_stateless/00921_datetime64_basic.sql index 1fc534d8afd..2d7cb975cfc 100644 --- a/tests/queries/0_stateless/00921_datetime64_basic.sql +++ b/tests/queries/0_stateless/00921_datetime64_basic.sql @@ -1,11 +1,11 @@ DROP TABLE IF EXISTS A; -SELECT CAST(1 as DateTime64('abc')); -- { serverError 42 } # Miss scale parameter type +SELECT CAST(1 as DateTime64('abc')); -- { serverError 43 } # Invalid scale parameter type SELECT CAST(1 as DateTime64(100)); -- { serverError 69 } # too big scale -SELECT CAST(1 as DateTime64(-1)); -- { serverError 42 } # Miss scale parameter type +SELECT CAST(1 as DateTime64(-1)); -- { serverError 43 } # signed scale parameter type SELECT CAST(1 as DateTime64(3, 'qqq')); -- { serverError 1000 } # invalid timezone -SELECT toDateTime64('2019-09-16 19:20:11.234', 'abc'); -- { serverError 43 } # Miss scale parameter type +SELECT toDateTime64('2019-09-16 19:20:11.234', 'abc'); -- { serverError 43 } # invalid scale SELECT toDateTime64('2019-09-16 19:20:11.234', 100); -- { serverError 69 } # too big scale SELECT toDateTime64(CAST([['CLb5Ph ']], 'String'), uniqHLL12('2Gs1V', 752)); -- { serverError 44 } # non-const string and non-const scale SELECT toDateTime64('2019-09-16 19:20:11.234', 3, 'qqq'); -- { serverError 1000 } # invalid timezone From e77ab608c8e579caca7131cc2036dbac3d32e582 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 19 Aug 2020 10:45:16 +0300 Subject: [PATCH 0068/1911] fix typo --- tests/integration/test_distributed_ddl/cluster.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_distributed_ddl/cluster.py b/tests/integration/test_distributed_ddl/cluster.py index b8c9527441e..258478de990 100644 --- a/tests/integration/test_distributed_ddl/cluster.py +++ b/tests/integration/test_distributed_ddl/cluster.py @@ -17,12 +17,12 @@ class ClickHouseClusterWithDDLHelpers(ClickHouseCluster): def prepare(self, replace_hostnames_with_ips=True): try: - main_configs_files = ["clusters.xml", "zookeeper_session_timeout.xml", "macro.xml", - "query_log.xml","ddl.xml"] + main_configs_files = ["clusters.xml", "zookeeper_session_timeout.xml", "macro.xml", "query_log.xml","ddl.xml"] main_configs = [os.path.join(self.test_config_dir, "config.d", f) for f in main_configs_files] user_configs = [os.path.join(self.test_config_dir, "users.d", f) for f in ["restricted_user.xml", "query_log.xml"]] if self.test_config_dir == "configs_secure": - main_configs += [os.path.join(self.test_config_dir, f) for i in ["server.crt", "server.key", "dhparam.pem", "config.d/ssl_conf.xml"]] + main_configs += [os.path.join(self.test_config_dir, f) for f in ["server.crt", "server.key", "dhparam.pem", "config.d/ssl_conf.xml"]] + for i in xrange(4): self.add_instance( 'ch{}'.format(i+1), From 31d31359f79ef7accd708f78149e44ee9e7ffc8b Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 19 Aug 2020 10:55:40 +0300 Subject: [PATCH 0069/1911] update dependencies --- docker/images.json | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/docker/images.json b/docker/images.json index e9e83f7859e..9a071afbed3 100644 --- a/docker/images.json +++ b/docker/images.json @@ -2,11 +2,7 @@ "docker/packager/deb": { "name": "yandex/clickhouse-deb-builder", "dependent": [ - "docker/packager/unbundled", - "docker/test/stateless", - "docker/test/stateless_with_coverage", - "docker/test/stateless_pytest", - "docker/test/coverage" + "docker/packager/unbundled" ] }, "docker/packager/binary": { @@ -20,6 +16,15 @@ "name": "yandex/clickhouse-unbundled-builder", "dependent": [] }, + "docker/test/base": { + "name": "yandex/clickhouse-test-base", + "dependent": [ + "docker/test/stateless_with_coverage", + "docker/test/coverage", + "test/stateless_pytest", + "test/stateless" + ] + }, "docker/test/coverage": { "name": "yandex/clickhouse-coverage", "dependent": [] From e44975df3b44b5dbaac36256ff5d34225a7aa682 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 19 Aug 2020 23:18:25 +0800 Subject: [PATCH 0070/1911] ISSUES-4006 try fix test failure --- src/DataTypes/registerDataTypeDateTime.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index eceb531b892..70b89bf7545 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -101,7 +101,7 @@ static DataTypePtr create64(const ASTPtr & arguments) throw Exception("DateTime64 data type can optionally have two argument - scale and time zone name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); const auto scale = getArgument(arguments, 0, "scale", "DateTime64"); - const auto timezone = getArgument(arguments, !!scale, "timezone", "DateTime64"); + const auto timezone = getArgument(arguments, 1, "timezone", "DateTime64"); return std::make_shared(scale, timezone.value_or(String{})); } From edeb983eb0d93ec66351238f349ef09a472ae083 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 20 Aug 2020 19:18:29 +0800 Subject: [PATCH 0071/1911] ISSUES-4006 some refactor --- src/Functions/FunctionsConversion.h | 64 ++++++++++++++--------------- 1 file changed, 32 insertions(+), 32 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 9e5a781240d..5fbcce4bc59 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -994,6 +994,18 @@ DEFINE_NAME_TO_INTERVAL(Year) #undef DEFINE_NAME_TO_INTERVAL +template +static inline bool isDateTime64(const ColumnsWithTypeAndName &arguments) +{ + if constexpr (std::is_same_v) + return true; + else if constexpr (std::is_same_v) + { + return (arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3; + } + + return false; +} template class FunctionConvert : public IFunction @@ -1024,16 +1036,14 @@ public: FunctionArgumentDescriptors mandatory_args = {{"Value", nullptr, nullptr, nullptr}}; FunctionArgumentDescriptors optional_args; - if constexpr (to_decimal || to_datetime64) + if constexpr (to_decimal) { mandatory_args.push_back({"scale", &isNativeInteger, &isColumnConst, "const Integer"}); } - if constexpr (std::is_same_v) + if (!to_decimal && isDateTime64(arguments)) { - /// toDateTime(value, scale:Integer) - if ((arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3) - mandatory_args.push_back({"scale", &isNativeInteger, &isColumnConst, "const Integer"}); + mandatory_args.push_back({"scale", &isNativeInteger, &isColumnConst, "const Integer"}); } // toString(DateTime or DateTime64, [timezone: String]) @@ -1079,29 +1089,22 @@ public: UInt32 scale [[maybe_unused]] = DataTypeDateTime64::default_scale; // DateTime64 requires more arguments: scale and timezone. Since timezone is optional, scale should be first. - if constexpr (to_datetime64) + if (isDateTime64(arguments)) { timezone_arg_position += 1; scale = static_cast(arguments[1].column->get64(0)); - } - if constexpr (std::is_same_v) - { - /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 - if ((arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3) - { - timezone_arg_position += 1; - scale = static_cast(arguments[1].column->get64(0)); - if (scale != 0) /// toDateTime('xxxx-xx-xx xx:xx:xx', 0) return DateTime - return std::make_shared( - scale, extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); - } + if (to_datetime64 || scale != 0) /// toDateTime('xxxx-xx-xx xx:xx:xx', 0) return DateTime + return std::make_shared(scale, + extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); + + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); } if constexpr (std::is_same_v) return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); - else if constexpr (to_datetime64) - return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); + else if constexpr (std::is_same_v) + throw Exception("LOGICAL ERROR: It is a bug.", ErrorCodes::LOGICAL_ERROR); else return std::make_shared(); } @@ -1201,22 +1204,19 @@ private: return true; }; - if constexpr (std::is_same_v) + if (isDateTime64(block.getColumnsWithTypeAndName())) { /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 - if ((arguments.size() == 2 && isUnsignedInteger(block.getByPosition(arguments[1]).type)) || arguments.size() == 3) + const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]); + UInt32 scale = extractToDecimalScale(scale_column); + + if (scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64 { - const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]); - UInt32 scale = extractToDecimalScale(scale_column); + if (!callOnIndexAndDataType(from_type->getTypeId(), call)) + throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64 - { - if (!callOnIndexAndDataType(from_type->getTypeId(), call)) - throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return; - } + return; } } From 6f6f3cb05f2c29143bd9d0309dd879bb363c9f29 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 20 Aug 2020 17:30:35 +0300 Subject: [PATCH 0072/1911] fix --- docker/images.json | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/images.json b/docker/images.json index 9a071afbed3..9b6c3ba3eed 100644 --- a/docker/images.json +++ b/docker/images.json @@ -21,8 +21,8 @@ "dependent": [ "docker/test/stateless_with_coverage", "docker/test/coverage", - "test/stateless_pytest", - "test/stateless" + "docker/test/stateless_pytest", + "docker/test/stateless" ] }, "docker/test/coverage": { From 45cc0778a0a65204e3c49653c7db067fa9fc1744 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 20 Aug 2020 22:41:03 +0800 Subject: [PATCH 0073/1911] ISSUES-4006 support scale with parserDateTime --- src/Functions/FunctionsConversion.h | 92 ++++++++++++++++++----------- 1 file changed, 59 insertions(+), 33 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 5fbcce4bc59..e4b990b53f4 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -994,14 +994,22 @@ DEFINE_NAME_TO_INTERVAL(Year) #undef DEFINE_NAME_TO_INTERVAL +struct NameParseDateTimeBestEffort; +struct NameParseDateTimeBestEffortOrZero; +struct NameParseDateTimeBestEffortOrNull; + template -static inline bool isDateTime64(const ColumnsWithTypeAndName &arguments) +static inline bool isDateTime64(const ColumnsWithTypeAndName & arguments, const ColumnNumbers & arguments_index = {}) { if constexpr (std::is_same_v) return true; - else if constexpr (std::is_same_v) + else if constexpr (std::is_same_v || std::is_same_v + || std::is_same_v || std::is_same_v) { - return (arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3; + if (arguments_index.empty()) + return (arguments.size() == 2 && isUnsignedInteger(arguments[1].type)) || arguments.size() == 3; + else + return (arguments_index.size() == 2 && isUnsignedInteger(arguments[arguments_index[1]].type)) || arguments_index.size() == 3; } return false; @@ -1204,7 +1212,7 @@ private: return true; }; - if (isDateTime64(block.getColumnsWithTypeAndName())) + if (isDateTime64(block.getColumnsWithTypeAndName(), arguments)) { /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]); @@ -1273,7 +1281,8 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { DataTypePtr res; - if constexpr (to_datetime64) + + if (isDateTime64(arguments)) { validateFunctionArgumentTypes(*this, arguments, FunctionArgumentDescriptors{{"string", isStringOrFixedString, nullptr, "String or FixedString"}}, @@ -1283,11 +1292,12 @@ public: {"timezone", isStringOrFixedString, isColumnConst, "const String or FixedString"}, }); - UInt64 scale = DataTypeDateTime64::default_scale; + UInt64 scale = to_datetime64 ? DataTypeDateTime64::default_scale : 0; if (arguments.size() > 1) scale = extractToDecimalScale(arguments[1]); const auto timezone = extractTimeZoneNameFromFunctionArguments(arguments, 2, 0); - res = std::make_shared(scale, timezone); + + res = scale == 0 ? res = std::make_shared(timezone) : std::make_shared(scale, timezone); } else { @@ -1334,6 +1344,8 @@ public: if constexpr (std::is_same_v) res = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0)); + else if constexpr (std::is_same_v) + throw Exception("LOGICAL ERROR: It is a bug.", ErrorCodes::LOGICAL_ERROR); else if constexpr (to_decimal) { UInt64 scale = extractToDecimalScale(arguments[1]); @@ -1358,42 +1370,53 @@ public: return res; } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override + template + bool executeInternal(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, UInt32 scale = 0) const { const IDataType * from_type = block.getByPosition(arguments[0]).type.get(); - bool ok = true; - if constexpr (to_decimal || to_datetime64) + if (checkAndGetDataType(from_type)) { - const UInt32 scale = assert_cast(*removeNullable(block.getByPosition(result).type)).getScale(); - - if (checkAndGetDataType(from_type)) - { - ConvertThroughParsing::execute( - block, arguments, result, input_rows_count, scale); - } - else if (checkAndGetDataType(from_type)) - { - ConvertThroughParsing::execute( - block, arguments, result, input_rows_count, scale); - } - else - ok = false; + ConvertThroughParsing::execute( + block, arguments, result, input_rows_count, scale); + return true; } + else if (checkAndGetDataType(from_type)) + { + ConvertThroughParsing::execute( + block, arguments, result, input_rows_count, scale); + return true; + } + + return false; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override + { + bool ok = true; + + if constexpr (to_decimal) + ok = executeInternal(block, arguments, result, input_rows_count, + assert_cast(*removeNullable(block.getByPosition(result).type)).getScale()); else { - if (checkAndGetDataType(from_type)) + if (isDateTime64(block.getColumnsWithTypeAndName(), arguments)) { - ConvertThroughParsing::execute( - block, arguments, result, input_rows_count); - } - else if (checkAndGetDataType(from_type)) - { - ConvertThroughParsing::execute( - block, arguments, result, input_rows_count); + UInt64 scale = to_datetime64 ? DataTypeDateTime64::default_scale : 0; + if (arguments.size() > 1) + scale = extractToDecimalScale(block.getColumnsWithTypeAndName()[arguments[1]]); + + if (scale == 0) + ok = executeInternal(block, arguments, result, input_rows_count); + else + { + ok = executeInternal(block, arguments, result, input_rows_count, static_cast(scale)); + } } else - ok = false; + { + ok = executeInternal(block, arguments, result, input_rows_count); + } } if (!ok) @@ -1757,6 +1780,9 @@ struct NameParseDateTimeBestEffort { static constexpr auto name = "parseDateTime struct NameParseDateTimeBestEffortUS { static constexpr auto name = "parseDateTimeBestEffortUS"; }; struct NameParseDateTimeBestEffortOrZero { static constexpr auto name = "parseDateTimeBestEffortOrZero"; }; struct NameParseDateTimeBestEffortOrNull { static constexpr auto name = "parseDateTimeBestEffortOrNull"; }; +struct NameParseDateTime32BestEffort { static constexpr auto name = "parseDateTime32BestEffort"; }; +struct NameParseDateTime32BestEffortOrZero { static constexpr auto name = "parseDateTime32BestEffortOrZero"; }; +struct NameParseDateTime32BestEffortOrNull { static constexpr auto name = "parseDateTime32BestEffortOrNull"; }; struct NameParseDateTime64BestEffort { static constexpr auto name = "parseDateTime64BestEffort"; }; struct NameParseDateTime64BestEffortOrZero { static constexpr auto name = "parseDateTime64BestEffortOrZero"; }; struct NameParseDateTime64BestEffortOrNull { static constexpr auto name = "parseDateTime64BestEffortOrNull"; }; From ec1572d7be7edc35d044dac603af2544b381b17e Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 21 Aug 2020 13:06:06 +0800 Subject: [PATCH 0074/1911] ISSUES-4006 support parserDateTime32 functions --- src/Functions/FunctionsConversion.cpp | 3 ++ src/Functions/FunctionsConversion.h | 7 +++ .../01442_date_time_with_params.reference | 40 +++++++++++++++ .../01442_date_time_with_params.sql | 50 +++++++++++++++++++ 4 files changed, 100 insertions(+) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 804c16d946d..428c6ba8138 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -82,6 +82,9 @@ void registerFunctionsConversion(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index e4b990b53f4..bcafcc3b59f 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1797,6 +1797,13 @@ using FunctionParseDateTimeBestEffortOrZero = FunctionConvertFromString< using FunctionParseDateTimeBestEffortOrNull = FunctionConvertFromString< DataTypeDateTime, NameParseDateTimeBestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>; +using FunctionParseDateTime32BestEffort = FunctionConvertFromString< + DataTypeDateTime, NameParseDateTime32BestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>; +using FunctionParseDateTime32BestEffortOrZero = FunctionConvertFromString< + DataTypeDateTime, NameParseDateTime32BestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>; +using FunctionParseDateTime32BestEffortOrNull = FunctionConvertFromString< + DataTypeDateTime, NameParseDateTime32BestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>; + using FunctionParseDateTime64BestEffort = FunctionConvertFromString< DataTypeDateTime64, NameParseDateTime64BestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>; using FunctionParseDateTime64BestEffortOrZero = FunctionConvertFromString< diff --git a/tests/queries/0_stateless/01442_date_time_with_params.reference b/tests/queries/0_stateless/01442_date_time_with_params.reference index 03b591a34bb..f38732b3f2f 100644 --- a/tests/queries/0_stateless/01442_date_time_with_params.reference +++ b/tests/queries/0_stateless/01442_date_time_with_params.reference @@ -2,3 +2,43 @@ 2020-01-01 00:00:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') 2020-01-01 00:05:00 DateTime 2020-01-01 00:00:00 DateTime 2020-01-01 00:02:00.11 DateTime64(2) 2020-01-01 00:03:00 DateTime(\'Europe/Moscow\') 2020-01-01 00:04:00.220 DateTime64(3, \'Europe/Moscow\') 2020-01-01 00:05:00 DateTime 2020-01-01 00:00:00 DateTime +2020-05-14 03:37:03.000 DateTime64(3, \'UTC\') +2020-05-14 03:37:03.000 DateTime64(3, \'UTC\') +2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') +2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') +2020-05-14 06:37:03.253 DateTime64(3, \'Europe/Minsk\') +2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') +\N Nullable(DateTime64(3)) +2020-05-14 03:37:03.000 Nullable(DateTime64(3, \'UTC\')) +2020-05-14 03:37:03.000 Nullable(DateTime64(3, \'UTC\')) +2020-05-14 03:37:03.253 Nullable(DateTime64(3, \'UTC\')) +2020-05-14 03:37:03.253 Nullable(DateTime64(3, \'UTC\')) +2020-05-14 06:37:03.253 Nullable(DateTime64(3, \'Europe/Minsk\')) +2020-05-14 03:37:03.253 Nullable(DateTime64(3, \'UTC\')) +1970-01-01 08:00:00.000 DateTime64(3) +2020-05-14 03:37:03.000 DateTime64(3, \'UTC\') +2020-05-14 03:37:03.000 DateTime64(3, \'UTC\') +2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') +2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') +2020-05-14 06:37:03.253 DateTime64(3, \'Europe/Minsk\') +2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') +2020-05-14 03:37:03 DateTime(\'UTC\') +2020-05-14 03:37:03 DateTime(\'UTC\') +2020-05-14 03:37:03 DateTime(\'UTC\') +2020-05-14 03:37:03 DateTime(\'UTC\') +2020-05-14 06:37:03 DateTime(\'Europe/Minsk\') +2020-05-14 03:37:03 DateTime(\'UTC\') +\N Nullable(DateTime) +2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) +2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) +2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) +2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) +2020-05-14 06:37:03 Nullable(DateTime(\'Europe/Minsk\')) +2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) +1970-01-01 08:00:00 DateTime +2020-05-14 03:37:03 DateTime(\'UTC\') +2020-05-14 03:37:03 DateTime(\'UTC\') +2020-05-14 03:37:03 DateTime(\'UTC\') +2020-05-14 03:37:03 DateTime(\'UTC\') +2020-05-14 06:37:03 DateTime(\'Europe/Minsk\') +2020-05-14 03:37:03 DateTime(\'UTC\') diff --git a/tests/queries/0_stateless/01442_date_time_with_params.sql b/tests/queries/0_stateless/01442_date_time_with_params.sql index d2664a4e316..5ae7fe22699 100644 --- a/tests/queries/0_stateless/01442_date_time_with_params.sql +++ b/tests/queries/0_stateless/01442_date_time_with_params.sql @@ -12,4 +12,54 @@ SELECT CAST('2020-01-01 00:00:00', 'DateTime') AS a, toTypeName(a), CAST('2020-0 SELECT toDateTime32('2020-01-01 00:00:00') AS a, toTypeName(a); +SELECT parseDateTimeBestEffort('', 3) AS a, toTypeName(a); -- {serverError 6} +SELECT parseDateTimeBestEffort('2020-05-14T03:37:03', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffort('2020-05-14 03:37:03', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffort('2020-05-14T03:37:03.253184', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffort('2020-05-14T03:37:03.253184Z', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffort('2020-05-14T03:37:03.253184Z', 3, 'Europe/Minsk') AS a, toTypeName(a); +SELECT parseDateTimeBestEffort(materialize('2020-05-14T03:37:03.253184Z'), 3, 'UTC') AS a, toTypeName(a); + +SELECT parseDateTimeBestEffortOrNull('', 3) AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrNull('2020-05-14T03:37:03', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrNull('2020-05-14 03:37:03', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrNull('2020-05-14T03:37:03.253184', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrNull('2020-05-14T03:37:03.253184Z', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrNull('2020-05-14T03:37:03.253184Z', 3, 'Europe/Minsk') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrNull(materialize('2020-05-14T03:37:03.253184Z'), 3, 'UTC') AS a, toTypeName(a); + +SELECT parseDateTimeBestEffortOrZero('', 3) AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrZero('2020-05-14T03:37:03', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrZero('2020-05-14 03:37:03', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrZero('2020-05-14T03:37:03.253184', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrZero('2020-05-14T03:37:03.253184Z', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrZero('2020-05-14T03:37:03.253184Z', 3, 'Europe/Minsk') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrZero(materialize('2020-05-14T03:37:03.253184Z'), 3, 'UTC') AS a, toTypeName(a); + + +SELECT parseDateTime32BestEffort('') AS a, toTypeName(a); -- {serverError 6} +SELECT parseDateTime32BestEffort('2020-05-14T03:37:03', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffort('2020-05-14 03:37:03', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffort('2020-05-14T03:37:03.253184', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffort('2020-05-14T03:37:03.253184Z', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffort('2020-05-14T03:37:03.253184Z', 'Europe/Minsk') AS a, toTypeName(a); +SELECT parseDateTime32BestEffort(materialize('2020-05-14T03:37:03.253184Z'), 'UTC') AS a, toTypeName(a); + +SELECT parseDateTime32BestEffortOrNull('') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrNull('2020-05-14T03:37:03', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrNull('2020-05-14 03:37:03', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrNull('2020-05-14T03:37:03.253184', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrNull('2020-05-14T03:37:03.253184Z', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrNull('2020-05-14T03:37:03.253184Z', 'Europe/Minsk') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrNull(materialize('2020-05-14T03:37:03.253184Z'), 'UTC') AS a, toTypeName(a); + +SELECT parseDateTime32BestEffortOrZero('') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrZero('2020-05-14T03:37:03', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrZero('2020-05-14 03:37:03', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrZero('2020-05-14T03:37:03.253184', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrZero('2020-05-14T03:37:03.253184Z', 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrZero('2020-05-14T03:37:03.253184Z', 'Europe/Minsk') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrZero(materialize('2020-05-14T03:37:03.253184Z'), 'UTC') AS a, toTypeName(a); + + DROP TABLE IF EXISTS test; From 3318b6ea00c478a0986d1fe526c172860dac1997 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 21 Aug 2020 13:08:45 +0800 Subject: [PATCH 0075/1911] ISSUES-4006 try fix build failure --- src/DataTypes/registerDataTypeDateTime.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index 70b89bf7545..815948c6531 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -48,7 +48,7 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume return {}; else { - if (argument->value.getType() != field_type) + if (argument && argument->value.getType() != field_type) throw Exception(getExceptionMessage(String(" has wrong type: ") + argument->value.getTypeName(), argument_index, argument_name, context_data_type_name, field_type), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); else From b679b2e30cdf01170352de3007880a01834341b7 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 21 Aug 2020 13:16:50 +0800 Subject: [PATCH 0076/1911] ISSUES-4006 fix toDateTime64 with scale 0 --- src/Functions/FunctionsConversion.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index bcafcc3b59f..5539d73d2eb 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1218,7 +1218,7 @@ private: const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]); UInt32 scale = extractToDecimalScale(scale_column); - if (scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64 + if (to_datetime64 || scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64 { if (!callOnIndexAndDataType(from_type->getTypeId(), call)) throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), From ff84040cd5394516a64688fc2701472325c00be6 Mon Sep 17 00:00:00 2001 From: Winter Zhang Date: Fri, 21 Aug 2020 14:42:31 +0800 Subject: [PATCH 0077/1911] ISSUES-4006 try fix test failure --- .../queries/0_stateless/01442_date_time_with_params.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01442_date_time_with_params.reference b/tests/queries/0_stateless/01442_date_time_with_params.reference index f38732b3f2f..f55d095d164 100644 --- a/tests/queries/0_stateless/01442_date_time_with_params.reference +++ b/tests/queries/0_stateless/01442_date_time_with_params.reference @@ -15,7 +15,7 @@ 2020-05-14 03:37:03.253 Nullable(DateTime64(3, \'UTC\')) 2020-05-14 06:37:03.253 Nullable(DateTime64(3, \'Europe/Minsk\')) 2020-05-14 03:37:03.253 Nullable(DateTime64(3, \'UTC\')) -1970-01-01 08:00:00.000 DateTime64(3) +1970-01-01 03:00:00.000 DateTime64(3) 2020-05-14 03:37:03.000 DateTime64(3, \'UTC\') 2020-05-14 03:37:03.000 DateTime64(3, \'UTC\') 2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') @@ -35,7 +35,7 @@ 2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) 2020-05-14 06:37:03 Nullable(DateTime(\'Europe/Minsk\')) 2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) -1970-01-01 08:00:00 DateTime +1970-01-01 03:00:00 DateTime 2020-05-14 03:37:03 DateTime(\'UTC\') 2020-05-14 03:37:03 DateTime(\'UTC\') 2020-05-14 03:37:03 DateTime(\'UTC\') From 2a96151516008a7b338346d87a6c88151cc95dae Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 22 Aug 2020 01:14:34 +0300 Subject: [PATCH 0078/1911] Fix GRANT ALL statement when executed on a non-global level. --- src/Access/AccessFlags.h | 29 +++++++++ src/Access/AccessRights.cpp | 75 ++++++---------------- src/Access/AccessRightsElement.h | 46 +++++++++++++ src/Interpreters/InterpreterGrantQuery.cpp | 2 +- src/Parsers/ASTGrantQuery.cpp | 8 +++ src/Parsers/ASTGrantQuery.h | 1 + src/Parsers/ParserGrantQuery.cpp | 27 ++++++++ 7 files changed, 131 insertions(+), 57 deletions(-) diff --git a/src/Access/AccessFlags.h b/src/Access/AccessFlags.h index 9b801fd88a3..11d39585238 100644 --- a/src/Access/AccessFlags.h +++ b/src/Access/AccessFlags.h @@ -96,6 +96,22 @@ public: /// Returns all the flags related to a dictionary. static AccessFlags allDictionaryFlags(); + /// Returns all the flags which could be granted on the global level. + /// The same as allFlags(). + static AccessFlags allFlagsGrantableOnGlobalLevel(); + + /// Returns all the flags which could be granted on the global level. + /// Returns allDatabaseFlags() | allTableFlags() | allDictionaryFlags() | allColumnFlags(). + static AccessFlags allFlagsGrantableOnDatabaseLevel(); + + /// Returns all the flags which could be granted on the table level. + /// Returns allTableFlags() | allDictionaryFlags() | allColumnFlags(). + static AccessFlags allFlagsGrantableOnTableLevel(); + + /// Returns all the flags which could be granted on the global level. + /// The same as allColumnFlags(). + static AccessFlags allFlagsGrantableOnColumnLevel(); + private: static constexpr size_t NUM_FLAGS = 128; using Flags = std::bitset; @@ -193,6 +209,10 @@ public: const Flags & getTableFlags() const { return all_flags_for_target[TABLE]; } const Flags & getColumnFlags() const { return all_flags_for_target[COLUMN]; } const Flags & getDictionaryFlags() const { return all_flags_for_target[DICTIONARY]; } + const Flags & getAllFlagsGrantableOnGlobalLevel() const { return getAllFlags(); } + const Flags & getAllFlagsGrantableOnDatabaseLevel() const { return all_flags_grantable_on_database_level; } + const Flags & getAllFlagsGrantableOnTableLevel() const { return all_flags_grantable_on_table_level; } + const Flags & getAllFlagsGrantableOnColumnLevel() const { return getColumnFlags(); } private: enum NodeType @@ -381,6 +401,9 @@ private: } for (const auto & child : start_node->children) collectAllFlags(child.get()); + + all_flags_grantable_on_table_level = all_flags_for_target[TABLE] | all_flags_for_target[DICTIONARY] | all_flags_for_target[COLUMN]; + all_flags_grantable_on_database_level = all_flags_for_target[DATABASE] | all_flags_grantable_on_table_level; } Impl() @@ -431,6 +454,8 @@ private: std::vector access_type_to_flags_mapping; Flags all_flags; Flags all_flags_for_target[static_cast(DICTIONARY) + 1]; + Flags all_flags_grantable_on_database_level; + Flags all_flags_grantable_on_table_level; }; @@ -447,6 +472,10 @@ inline AccessFlags AccessFlags::allDatabaseFlags() { return Impl<>::instance().g inline AccessFlags AccessFlags::allTableFlags() { return Impl<>::instance().getTableFlags(); } inline AccessFlags AccessFlags::allColumnFlags() { return Impl<>::instance().getColumnFlags(); } inline AccessFlags AccessFlags::allDictionaryFlags() { return Impl<>::instance().getDictionaryFlags(); } +inline AccessFlags AccessFlags::allFlagsGrantableOnGlobalLevel() { return Impl<>::instance().getAllFlagsGrantableOnGlobalLevel(); } +inline AccessFlags AccessFlags::allFlagsGrantableOnDatabaseLevel() { return Impl<>::instance().getAllFlagsGrantableOnDatabaseLevel(); } +inline AccessFlags AccessFlags::allFlagsGrantableOnTableLevel() { return Impl<>::instance().getAllFlagsGrantableOnTableLevel(); } +inline AccessFlags AccessFlags::allFlagsGrantableOnColumnLevel() { return Impl<>::instance().getAllFlagsGrantableOnColumnLevel(); } inline AccessFlags operator |(AccessType left, AccessType right) { return AccessFlags(left) | right; } inline AccessFlags operator &(AccessType left, AccessType right) { return AccessFlags(left) & right; } diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 65c78f39e86..8ce71dd8da8 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include @@ -8,12 +7,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int INVALID_GRANT; -} - - namespace { using Kind = AccessRightsElementWithOptions::Kind; @@ -214,30 +207,14 @@ namespace COLUMN_LEVEL, }; - AccessFlags getAcceptableFlags(Level level) + AccessFlags getAllGrantableFlags(Level level) { switch (level) { - case GLOBAL_LEVEL: - { - static const AccessFlags res = AccessFlags::allFlags(); - return res; - } - case DATABASE_LEVEL: - { - static const AccessFlags res = AccessFlags::allDatabaseFlags() | AccessFlags::allTableFlags() | AccessFlags::allDictionaryFlags() | AccessFlags::allColumnFlags(); - return res; - } - case TABLE_LEVEL: - { - static const AccessFlags res = AccessFlags::allTableFlags() | AccessFlags::allDictionaryFlags() | AccessFlags::allColumnFlags(); - return res; - } - case COLUMN_LEVEL: - { - static const AccessFlags res = AccessFlags::allColumnFlags(); - return res; - } + case GLOBAL_LEVEL: return AccessFlags::allFlagsGrantableOnGlobalLevel(); + case DATABASE_LEVEL: return AccessFlags::allFlagsGrantableOnDatabaseLevel(); + case TABLE_LEVEL: return AccessFlags::allFlagsGrantableOnTableLevel(); + case COLUMN_LEVEL: return AccessFlags::allFlagsGrantableOnColumnLevel(); } __builtin_unreachable(); } @@ -276,21 +253,7 @@ public: void grant(const AccessFlags & flags_) { - if (!flags_) - return; - - AccessFlags flags_to_add = flags_ & getAcceptableFlags(); - - if (!flags_to_add) - { - if (level == DATABASE_LEVEL) - throw Exception(flags_.toString() + " cannot be granted on the database level", ErrorCodes::INVALID_GRANT); - else if (level == TABLE_LEVEL) - throw Exception(flags_.toString() + " cannot be granted on the table level", ErrorCodes::INVALID_GRANT); - else if (level == COLUMN_LEVEL) - throw Exception(flags_.toString() + " cannot be granted on the column level", ErrorCodes::INVALID_GRANT); - } - + AccessFlags flags_to_add = flags_ & getAllGrantableFlags(); addGrantsRec(flags_to_add); optimizeTree(); } @@ -456,8 +419,8 @@ public: } private: - AccessFlags getAcceptableFlags() const { return ::DB::getAcceptableFlags(level); } - AccessFlags getChildAcceptableFlags() const { return ::DB::getAcceptableFlags(static_cast(level + 1)); } + AccessFlags getAllGrantableFlags() const { return ::DB::getAllGrantableFlags(level); } + AccessFlags getChildAllGrantableFlags() const { return ::DB::getAllGrantableFlags(static_cast(level + 1)); } Node * tryGetChild(const std::string_view & name) const { @@ -480,7 +443,7 @@ private: Node & new_child = (*children)[*new_child_name]; new_child.node_name = std::move(new_child_name); new_child.level = static_cast(level + 1); - new_child.flags = flags & new_child.getAcceptableFlags(); + new_child.flags = flags & new_child.getAllGrantableFlags(); return new_child; } @@ -496,12 +459,12 @@ private: bool canEraseChild(const Node & child) const { - return ((flags & child.getAcceptableFlags()) == child.flags) && !child.children; + return ((flags & child.getAllGrantableFlags()) == child.flags) && !child.children; } void addGrantsRec(const AccessFlags & flags_) { - if (auto flags_to_add = flags_ & getAcceptableFlags()) + if (auto flags_to_add = flags_ & getAllGrantableFlags()) { flags |= flags_to_add; if (children) @@ -547,7 +510,7 @@ private: const AccessFlags & parent_flags) { auto flags = node.flags; - auto parent_fl = parent_flags & node.getAcceptableFlags(); + auto parent_fl = parent_flags & node.getAllGrantableFlags(); auto revokes = parent_fl - flags; auto grants = flags - parent_fl; @@ -576,9 +539,9 @@ private: const Node * node_go, const AccessFlags & parent_flags_go) { - auto acceptable_flags = ::DB::getAcceptableFlags(static_cast(full_name.size())); - auto parent_fl = parent_flags & acceptable_flags; - auto parent_fl_go = parent_flags_go & acceptable_flags; + auto grantable_flags = ::DB::getAllGrantableFlags(static_cast(full_name.size())); + auto parent_fl = parent_flags & grantable_flags; + auto parent_fl_go = parent_flags_go & grantable_flags; auto flags = node ? node->flags : parent_fl; auto flags_go = node_go ? node_go->flags : parent_fl_go; auto revokes = parent_fl - flags; @@ -672,8 +635,8 @@ private: } max_flags_with_children |= max_among_children; - AccessFlags add_acceptable_flags = getAcceptableFlags() - getChildAcceptableFlags(); - min_flags_with_children &= min_among_children | add_acceptable_flags; + AccessFlags add_flags = getAllGrantableFlags() - getChildAllGrantableFlags(); + min_flags_with_children &= min_among_children | add_flags; } void makeUnionRec(const Node & rhs) @@ -689,7 +652,7 @@ private: for (auto & [lhs_childname, lhs_child] : *children) { if (!rhs.tryGetChild(lhs_childname)) - lhs_child.flags |= rhs.flags & lhs_child.getAcceptableFlags(); + lhs_child.flags |= rhs.flags & lhs_child.getAllGrantableFlags(); } } } @@ -738,7 +701,7 @@ private: if (new_flags != flags) { - new_flags &= getAcceptableFlags(); + new_flags &= getAllGrantableFlags(); flags_added |= static_cast(new_flags - flags); flags_removed |= static_cast(flags - new_flags); flags = new_flags; diff --git a/src/Access/AccessRightsElement.h b/src/Access/AccessRightsElement.h index f9f7c433308..36cb64e6eba 100644 --- a/src/Access/AccessRightsElement.h +++ b/src/Access/AccessRightsElement.h @@ -71,6 +71,8 @@ struct AccessRightsElement { } + bool empty() const { return !access_flags || (!any_column && columns.empty()); } + auto toTuple() const { return std::tie(access_flags, any_database, database, any_table, table, any_column, columns); } friend bool operator==(const AccessRightsElement & left, const AccessRightsElement & right) { return left.toTuple() == right.toTuple(); } friend bool operator!=(const AccessRightsElement & left, const AccessRightsElement & right) { return !(left == right); } @@ -86,6 +88,9 @@ struct AccessRightsElement /// If the database is empty, replaces it with `new_database`. Otherwise does nothing. void replaceEmptyDatabase(const String & new_database); + /// Resets flags which cannot be granted. + void removeNonGrantableFlags(); + /// Returns a human-readable representation like "SELECT, UPDATE(x, y) ON db.table". String toString() const; }; @@ -111,6 +116,9 @@ struct AccessRightsElementWithOptions : public AccessRightsElement friend bool operator==(const AccessRightsElementWithOptions & left, const AccessRightsElementWithOptions & right) { return left.toTuple() == right.toTuple(); } friend bool operator!=(const AccessRightsElementWithOptions & left, const AccessRightsElementWithOptions & right) { return !(left == right); } + /// Resets flags which cannot be granted. + void removeNonGrantableFlags(); + /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; }; @@ -120,9 +128,14 @@ struct AccessRightsElementWithOptions : public AccessRightsElement class AccessRightsElements : public std::vector { public: + bool empty() const { return std::all_of(begin(), end(), [](const AccessRightsElement & e) { return e.empty(); }); } + /// Replaces the empty database with `new_database`. void replaceEmptyDatabase(const String & new_database); + /// Resets flags which cannot be granted. + void removeNonGrantableFlags(); + /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; }; @@ -134,6 +147,9 @@ public: /// Replaces the empty database with `new_database`. void replaceEmptyDatabase(const String & new_database); + /// Resets flags which cannot be granted. + void removeNonGrantableFlags(); + /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; }; @@ -157,4 +173,34 @@ inline void AccessRightsElementsWithOptions::replaceEmptyDatabase(const String & element.replaceEmptyDatabase(new_database); } +inline void AccessRightsElement::removeNonGrantableFlags() +{ + if (!any_column) + access_flags &= AccessFlags::allFlagsGrantableOnColumnLevel(); + else if (!any_table) + access_flags &= AccessFlags::allFlagsGrantableOnTableLevel(); + else if (!any_database) + access_flags &= AccessFlags::allFlagsGrantableOnDatabaseLevel(); + else + access_flags &= AccessFlags::allFlagsGrantableOnGlobalLevel(); +} + +inline void AccessRightsElementWithOptions::removeNonGrantableFlags() +{ + if (kind == Kind::GRANT) + AccessRightsElement::removeNonGrantableFlags(); +} + +inline void AccessRightsElements::removeNonGrantableFlags() +{ + for (auto & element : *this) + element.removeNonGrantableFlags(); +} + +inline void AccessRightsElementsWithOptions::removeNonGrantableFlags() +{ + for (auto & element : *this) + element.removeNonGrantableFlags(); +} + } diff --git a/src/Interpreters/InterpreterGrantQuery.cpp b/src/Interpreters/InterpreterGrantQuery.cpp index 2f468507eb6..57cb701036e 100644 --- a/src/Interpreters/InterpreterGrantQuery.cpp +++ b/src/Interpreters/InterpreterGrantQuery.cpp @@ -29,7 +29,6 @@ namespace current_access.grant(access_to_grant); } - AccessRightsElements getFilteredAccessRightsElementsToRevoke( const AccessRights & current_access, const AccessRightsElements & access_to_revoke, bool grant_option) { @@ -214,6 +213,7 @@ BlockIO InterpreterGrantQuery::execute() auto access = context.getAccess(); auto & access_control = context.getAccessControlManager(); query.replaceEmptyDatabaseWithCurrent(context.getCurrentDatabase()); + query.removeNonGrantableFlags(); RolesOrUsersSet roles_from_query; if (query.roles) diff --git a/src/Parsers/ASTGrantQuery.cpp b/src/Parsers/ASTGrantQuery.cpp index ae9649cdddc..63489e0417f 100644 --- a/src/Parsers/ASTGrantQuery.cpp +++ b/src/Parsers/ASTGrantQuery.cpp @@ -144,4 +144,12 @@ void ASTGrantQuery::replaceCurrentUserTagWithName(const String & current_user_na if (to_roles) to_roles->replaceCurrentUserTagWithName(current_user_name); } + + +void ASTGrantQuery::removeNonGrantableFlags() +{ + if (kind == Kind::GRANT) + access_rights_elements.removeNonGrantableFlags(); +} + } diff --git a/src/Parsers/ASTGrantQuery.h b/src/Parsers/ASTGrantQuery.h index c36e42689a5..5f172fe3298 100644 --- a/src/Parsers/ASTGrantQuery.h +++ b/src/Parsers/ASTGrantQuery.h @@ -33,6 +33,7 @@ public: void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void replaceEmptyDatabaseWithCurrent(const String & current_database); void replaceCurrentUserTagWithName(const String & current_user_name) const; + void removeNonGrantableFlags(); ASTPtr getRewrittenASTWithoutOnCluster(const std::string &) const override { return removeOnCluster(clone()); } }; } diff --git a/src/Parsers/ParserGrantQuery.cpp b/src/Parsers/ParserGrantQuery.cpp index 6e42b165b21..7dd721c9af2 100644 --- a/src/Parsers/ParserGrantQuery.cpp +++ b/src/Parsers/ParserGrantQuery.cpp @@ -14,6 +14,7 @@ namespace DB { namespace ErrorCodes { + extern const int INVALID_GRANT; extern const int SYNTAX_ERROR; } @@ -156,6 +157,29 @@ namespace } + void removeNonGrantableFlags(AccessRightsElements & elements) + { + for (auto & element : elements) + { + if (element.empty()) + continue; + auto old_flags = element.access_flags; + element.removeNonGrantableFlags(); + if (!element.empty()) + continue; + + if (!element.any_column) + throw Exception(old_flags.toString() + " cannot be granted on the column level", ErrorCodes::INVALID_GRANT); + else if (!element.any_table) + throw Exception(old_flags.toString() + " cannot be granted on the table level", ErrorCodes::INVALID_GRANT); + else if (!element.any_database) + throw Exception(old_flags.toString() + " cannot be granted on the database level", ErrorCodes::INVALID_GRANT); + else + throw Exception(old_flags.toString() + " cannot be granted", ErrorCodes::INVALID_GRANT); + } + } + + bool parseRoles(IParser::Pos & pos, Expected & expected, Kind kind, bool id_mode, std::shared_ptr & roles) { return IParserBase::wrapParseImpl(pos, [&] @@ -274,6 +298,9 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (admin_option && !elements.empty()) throw Exception("ADMIN OPTION should be specified for roles", ErrorCodes::SYNTAX_ERROR); + if (kind == Kind::GRANT) + removeNonGrantableFlags(elements); + auto query = std::make_shared(); node = query; From 4c8a8d5e67ec613f9d164366279e9e7b81577111 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 22 Aug 2020 01:37:01 +0300 Subject: [PATCH 0079/1911] Add test. --- tests/integration/test_grant_and_revoke/test.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index 92ffb78a1cb..1557e81bce8 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -107,6 +107,15 @@ def test_revoke_requires_grant_option(): assert instance.query("SHOW GRANTS FOR B") == "" +def test_grant_all_on_table(): + instance.query("CREATE USER A, B") + instance.query("GRANT ALL ON test.table TO A WITH GRANT OPTION") + instance.query("GRANT ALL ON test.table TO B", user='A') + assert instance.query("SHOW GRANTS FOR B") == "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, DROP TABLE, DROP VIEW, DROP DICTIONARY, TRUNCATE, OPTIMIZE, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON test.table TO B\n" + instance.query("REVOKE ALL ON test.table FROM B", user='A') + assert instance.query("SHOW GRANTS FOR B") == "" + + def test_implicit_show_grants(): instance.query("CREATE USER A") assert instance.query("select count() FROM system.databases WHERE name='test'", user="A") == "0\n" From 0e8cb96eb13491e778c54f0876d590002b8a2436 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 22 Aug 2020 22:57:46 +0800 Subject: [PATCH 0080/1911] Try fix IfAggCombinator with NullAggCombinator --- .../AggregateFunctionCount.cpp | 2 +- .../AggregateFunctionCount.h | 2 +- .../AggregateFunctionIf.cpp | 110 +++++++++++++++ src/AggregateFunctions/AggregateFunctionIf.h | 4 + .../AggregateFunctionNull.cpp | 2 +- .../AggregateFunctionNull.h | 127 ++++++++++++++---- .../AggregateFunctionWindowFunnel.h | 3 +- src/AggregateFunctions/IAggregateFunction.h | 4 +- ...able_type_with_if_agg_combinator.reference | 3 + ...5_nullable_type_with_if_agg_combinator.sql | 6 + 10 files changed, 229 insertions(+), 34 deletions(-) create mode 100644 tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.reference create mode 100644 tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql diff --git a/src/AggregateFunctions/AggregateFunctionCount.cpp b/src/AggregateFunctions/AggregateFunctionCount.cpp index 6ea63bedaf0..05824947b87 100644 --- a/src/AggregateFunctions/AggregateFunctionCount.cpp +++ b/src/AggregateFunctions/AggregateFunctionCount.cpp @@ -8,7 +8,7 @@ namespace DB { AggregateFunctionPtr AggregateFunctionCount::getOwnNullAdapter( - const AggregateFunctionPtr &, const DataTypes & types, const Array & params) const + const AggregateFunctionPtr &, const DataTypes & types, const Array & params, const AggregateFunctionProperties & /*properties*/) const { return std::make_shared(types[0], params); } diff --git a/src/AggregateFunctions/AggregateFunctionCount.h b/src/AggregateFunctions/AggregateFunctionCount.h index 29c5de0021c..eb1583df92a 100644 --- a/src/AggregateFunctions/AggregateFunctionCount.h +++ b/src/AggregateFunctions/AggregateFunctionCount.h @@ -69,7 +69,7 @@ public: } AggregateFunctionPtr getOwnNullAdapter( - const AggregateFunctionPtr &, const DataTypes & types, const Array & params) const override; + const AggregateFunctionPtr &, const DataTypes & types, const Array & params, const AggregateFunctionProperties & /*properties*/) const override; }; diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index 19a175de911..4722732a8ba 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -1,6 +1,7 @@ #include #include #include "registerAggregateFunctions.h" +#include "AggregateFunctionNull.h" namespace DB @@ -40,6 +41,115 @@ public: } }; +/** There are two cases: for single argument and variadic. + * Code for single argument is much more efficient. + */ +template +class AggregateFunctionIfNullUnary final + : public AggregateFunctionNullUnaryBase> +{ +private: + size_t num_arguments; + + using Base = AggregateFunctionNullUnaryBase>; +public: + + String getName() const override + { + return Base::getName() + "If"; + } + + AggregateFunctionIfNullUnary(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) + : AggregateFunctionNullUnaryBase>(std::move(nested_function_), arguments, params), + num_arguments(arguments.size()) + { + if (num_arguments == 0) + throw Exception("Aggregate function " + getName() + " require at least one argument", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + bool singleFilter(const IColumn ** columns, size_t row_num) const override + { + const IColumn * filter_column = columns[num_arguments - 1]; + if (const ColumnNullable * nullable_column = typeid_cast(filter_column)) + filter_column = nullable_column->getNestedColumnPtr().get(); + + return assert_cast(*filter_column).getData()[row_num]; + } +}; + +template +class AggregateFunctionIfNullVariadic final + : public AggregateFunctionNullVariadicBase> +{ +private: + size_t num_arguments; + + using Base = AggregateFunctionNullVariadicBase>; +public: + + String getName() const override + { + return Base::getName() + "If"; + } + + AggregateFunctionIfNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) + : AggregateFunctionNullVariadicBase>(std::move(nested_function_), arguments, params), + num_arguments(arguments.size()) + { + } + + bool singleFilter(const IColumn ** columns, size_t row_num) const override + { + return assert_cast(*columns[num_arguments - 1]).getData()[row_num]; + } +}; + + +AggregateFunctionPtr AggregateFunctionIf::getOwnNullAdapter( + const AggregateFunctionPtr & nested_function, const DataTypes & arguments, + const Array & params, const AggregateFunctionProperties & properties) const +{ + bool return_type_is_nullable = !properties.returns_default_when_only_null && getReturnType()->canBeInsideNullable(); + size_t nullable_size = std::count_if(arguments.begin(), arguments.end(), [](const auto & element) { return element->isNullable(); }); + return_type_is_nullable &= nullable_size != 1 || !arguments.back()->isNullable(); /// If only condition is nullable. we should non-nullable type. + bool serialize_flag = return_type_is_nullable || properties.returns_default_when_only_null; + + if (arguments.size() <= 2 && arguments.front()->isNullable()) + { + if (return_type_is_nullable) + { + return std::make_shared>(nested_func, arguments, params); + } + else + { + if (serialize_flag) + return std::make_shared>(nested_func, arguments, params); + else + return std::make_shared>(nested_func, arguments, params); + } + } + else + { + if (return_type_is_nullable) + { + return std::make_shared>(nested_function, arguments, params); + } + else + { + if (serialize_flag) + return std::make_shared>(nested_function, arguments, params); + else + return std::make_shared>(nested_function, arguments, params); + } + } +} + void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory & factory) { factory.registerCombinator(std::make_shared()); diff --git a/src/AggregateFunctions/AggregateFunctionIf.h b/src/AggregateFunctions/AggregateFunctionIf.h index f04450c9142..d5d2b9be0dd 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.h +++ b/src/AggregateFunctions/AggregateFunctionIf.h @@ -109,6 +109,10 @@ public: { return nested_func->isState(); } + + AggregateFunctionPtr getOwnNullAdapter( + const AggregateFunctionPtr & nested_function, const DataTypes & arguments, + const Array & params, const AggregateFunctionProperties & properties) const override; }; } diff --git a/src/AggregateFunctions/AggregateFunctionNull.cpp b/src/AggregateFunctions/AggregateFunctionNull.cpp index b65b4aba447..c88d1e7f24c 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.cpp +++ b/src/AggregateFunctions/AggregateFunctionNull.cpp @@ -69,7 +69,7 @@ public: assert(nested_function); - if (auto adapter = nested_function->getOwnNullAdapter(nested_function, arguments, params)) + if (auto adapter = nested_function->getOwnNullAdapter(nested_function, arguments, params, properties)) return adapter; /// If applied to aggregate function with -State combinator, we apply -Null combinator to it's nested_function instead of itself. diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index 2f34aef0932..a50e97a3166 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -183,29 +183,37 @@ public: }; -/** There are two cases: for single argument and variadic. - * Code for single argument is much more efficient. - */ -template -class AggregateFunctionNullUnary final - : public AggregateFunctionNullBase> +template +class AggregateFunctionNullUnaryBase : public AggregateFunctionNullBase { public: - AggregateFunctionNullUnary(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) - : AggregateFunctionNullBase>(std::move(nested_function_), arguments, params) + AggregateFunctionNullUnaryBase(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) + : AggregateFunctionNullBase(std::move(nested_function_), arguments, params) { } + virtual bool singleFilter(const IColumn ** columns, size_t row_num) const = 0; + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override { const ColumnNullable * column = assert_cast(columns[0]); const IColumn * nested_column = &column->getNestedColumn(); - if (!column->isNullAt(row_num)) + + if constexpr (other_filter) { - this->setFlag(place); - this->nested_function->add(this->nestedPlace(place), &nested_column, row_num, arena); + if (!column->isNullAt(row_num) && singleFilter(columns, row_num)) + { + this->setFlag(place); + this->nested_function->add(this->nestedPlace(place), &nested_column, row_num, arena); + } + } + else + { + if (!column->isNullAt(row_num)) + { + this->setFlag(place); + this->nested_function->add(this->nestedPlace(place), &nested_column, row_num, arena); + } } } @@ -215,25 +223,58 @@ public: const IColumn * nested_column = &column->getNestedColumn(); const UInt8 * null_map = column->getNullMapData().data(); - this->nested_function->addBatchSinglePlaceNotNull(batch_size, this->nestedPlace(place), &nested_column, null_map, arena); - if constexpr (result_is_nullable) - if (!memoryIsByte(null_map, batch_size, 1)) - this->setFlag(place); + { + if constexpr (other_filter) + { + bool flag = false; + for (size_t i = 0; i < batch_size; ++i) + { + if (!null_map[i] && singleFilter(columns, i)) + { + flag = true; + static_cast(this)->add(place, columns, i, arena); + } + } + + if (flag) + this->setFlag(place); + } + else + { + this->nested_function->addBatchSinglePlaceNotNull(batch_size, this->nestedPlace(place), &nested_column, null_map, arena); + + if (!memoryIsByte(null_map, batch_size, 1)) + this->setFlag(place); + } + } } }; - -template -class AggregateFunctionNullVariadic final - : public AggregateFunctionNullBase> +/** There are two cases: for single argument and variadic. + * Code for single argument is much more efficient. + */ +template +class AggregateFunctionNullUnary final + : public AggregateFunctionNullUnaryBase> { public: - AggregateFunctionNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) - : AggregateFunctionNullBase>(std::move(nested_function_), arguments, params), - number_of_arguments(arguments.size()) + AggregateFunctionNullUnary(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) + : AggregateFunctionNullUnaryBase>(std::move(nested_function_), arguments, params) + { + } + + bool singleFilter(const IColumn ** /*columns*/, size_t /*row_num*/) const override { return true; } +}; + +template +class AggregateFunctionNullVariadicBase : public AggregateFunctionNullBase +{ +public: + AggregateFunctionNullVariadicBase(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) + : AggregateFunctionNullBase(std::move(nested_function_), arguments, params), + number_of_arguments(arguments.size()) { if (number_of_arguments == 1) throw Exception("Logical error: single argument is passed to AggregateFunctionNullVariadic", ErrorCodes::LOGICAL_ERROR); @@ -246,6 +287,8 @@ public: is_nullable[i] = arguments[i]->isNullable(); } + virtual bool singleFilter(const IColumn ** columns, size_t row_num) const = 0; + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override { /// This container stores the columns we really pass to the nested function. @@ -268,14 +311,40 @@ public: nested_columns[i] = columns[i]; } - this->setFlag(place); - this->nested_function->add(this->nestedPlace(place), nested_columns, row_num, arena); + if constexpr (other_filter) + { + if (singleFilter(nested_columns, row_num)) + { + this->setFlag(place); + this->nested_function->add(this->nestedPlace(place), nested_columns, row_num, arena); + } + } + else + { + this->setFlag(place); + this->nested_function->add(this->nestedPlace(place), nested_columns, row_num, arena); + } } -private: +protected: enum { MAX_ARGS = 8 }; size_t number_of_arguments = 0; std::array is_nullable; /// Plain array is better than std::vector due to one indirection less. }; +template +class AggregateFunctionNullVariadic final + : public AggregateFunctionNullVariadicBase> +{ +public: + AggregateFunctionNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) + : AggregateFunctionNullVariadicBase>(std::move(nested_function_), arguments, params) + { + } + + bool singleFilter(const IColumn ** /*columns*/, size_t /*row_num*/) const override { return true; } +}; + } diff --git a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index 3297819a9ff..fe45fec4b76 100644 --- a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -241,7 +241,8 @@ public: } AggregateFunctionPtr getOwnNullAdapter( - const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params) const override + const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, + const AggregateFunctionProperties & /*properties*/) const override { return std::make_shared>(nested_function, arguments, params); } diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index d9c619809fc..7e6b7abbd28 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -33,6 +33,7 @@ using ConstAggregateDataPtr = const char *; class IAggregateFunction; using AggregateFunctionPtr = std::shared_ptr; +struct AggregateFunctionProperties; /** Aggregate functions interface. * Instances of classes with this interface do not contain the data itself for aggregation, @@ -186,7 +187,8 @@ public: * arguments and params are for nested_function. */ virtual AggregateFunctionPtr getOwnNullAdapter( - const AggregateFunctionPtr & /*nested_function*/, const DataTypes & /*arguments*/, const Array & /*params*/) const + const AggregateFunctionPtr & /*nested_function*/, const DataTypes & /*arguments*/, + const Array & /*params*/, const AggregateFunctionProperties & /*properties*/) const { return nullptr; } diff --git a/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.reference b/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.reference new file mode 100644 index 00000000000..77f38b722ce --- /dev/null +++ b/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.reference @@ -0,0 +1,3 @@ +\N Nullable(UInt8) +\N Nullable(UInt8) +0 UInt8 diff --git a/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql b/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql new file mode 100644 index 00000000000..852660117f5 --- /dev/null +++ b/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql @@ -0,0 +1,6 @@ +-- Value nullable +SELECT anyIf(CAST(number, 'Nullable(UInt8)'), number = 3) AS a, toTypeName(a) FROM numbers(2); +-- Value and condition nullable +SELECT anyIf(number, number = 3) AS a, toTypeName(a) FROM (SELECT CAST(number, 'Nullable(UInt8)') AS number FROM numbers(2)); +-- Condition nullable +SELECT anyIf(CAST(number, 'UInt8'), number = 3) AS a, toTypeName(a) FROM (SELECT CAST(number, 'Nullable(UInt8)') AS number FROM numbers(2)); From 63868aaed3ac7dbb130d69c6c65a42ab6f2c8348 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 24 Aug 2020 13:05:59 +0800 Subject: [PATCH 0081/1911] Try fix CI --- .../AggregateFunctionIf.cpp | 81 ++++++++--- .../AggregateFunctionNull.h | 127 ++++-------------- 2 files changed, 94 insertions(+), 114 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index 4722732a8ba..8d10fda5f27 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -46,12 +46,13 @@ public: */ template class AggregateFunctionIfNullUnary final - : public AggregateFunctionNullUnaryBase> + : public AggregateFunctionNullBase> { private: size_t num_arguments; - using Base = AggregateFunctionNullUnaryBase>; public: @@ -61,16 +62,14 @@ public: } AggregateFunctionIfNullUnary(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) - : AggregateFunctionNullUnaryBase>(std::move(nested_function_), arguments, params), - num_arguments(arguments.size()) + : Base(std::move(nested_function_), arguments, params), num_arguments(arguments.size()) { if (num_arguments == 0) throw Exception("Aggregate function " + getName() + " require at least one argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } - bool singleFilter(const IColumn ** columns, size_t row_num) const override + static inline bool singleFilter(const IColumn ** columns, size_t row_num, size_t num_arguments) { const IColumn * filter_column = columns[num_arguments - 1]; if (const ColumnNullable * nullable_column = typeid_cast(filter_column)) @@ -78,18 +77,24 @@ public: return assert_cast(*filter_column).getData()[row_num]; } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + const ColumnNullable * column = assert_cast(columns[0]); + const IColumn * nested_column = &column->getNestedColumn(); + if (!column->isNullAt(row_num) && singleFilter(columns, row_num, num_arguments)) + { + this->setFlag(place); + this->nested_function->add(this->nestedPlace(place), &nested_column, row_num, arena); + } + } }; template class AggregateFunctionIfNullVariadic final - : public AggregateFunctionNullVariadicBase> { -private: - size_t num_arguments; - - using Base = AggregateFunctionNullVariadicBase>; public: String getName() const override @@ -98,16 +103,60 @@ public: } AggregateFunctionIfNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) - : AggregateFunctionNullVariadicBase>(std::move(nested_function_), arguments, params), - num_arguments(arguments.size()) + : Base(std::move(nested_function_), arguments, params), number_of_arguments(arguments.size()) { + if (number_of_arguments == 1) + throw Exception("Logical error: single argument is passed to AggregateFunctionIfNullVariadic", ErrorCodes::LOGICAL_ERROR); + + if (number_of_arguments > MAX_ARGS) + throw Exception("Maximum number of arguments for aggregate function with Nullable types is " + toString(size_t(MAX_ARGS)), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (size_t i = 0; i < number_of_arguments; ++i) + is_nullable[i] = arguments[i]->isNullable(); } - bool singleFilter(const IColumn ** columns, size_t row_num) const override + static inline bool singleFilter(const IColumn ** columns, size_t row_num, size_t num_arguments) { return assert_cast(*columns[num_arguments - 1]).getData()[row_num]; } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + /// This container stores the columns we really pass to the nested function. + const IColumn * nested_columns[number_of_arguments]; + + for (size_t i = 0; i < number_of_arguments; ++i) + { + if (is_nullable[i]) + { + const ColumnNullable & nullable_col = assert_cast(*columns[i]); + if (null_is_skipped && nullable_col.isNullAt(row_num)) + { + /// If at least one column has a null value in the current row, + /// we don't process this row. + return; + } + nested_columns[i] = &nullable_col.getNestedColumn(); + } + else + nested_columns[i] = columns[i]; + } + + if (singleFilter(nested_columns, row_num, number_of_arguments)) + { + this->setFlag(place); + this->nested_function->add(this->nestedPlace(place), nested_columns, row_num, arena); + } + } + +private: + using Base = AggregateFunctionNullBase>; + + enum { MAX_ARGS = 8 }; + size_t number_of_arguments = 0; + std::array is_nullable; /// Plain array is better than std::vector due to one indirection less. }; diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index a50e97a3166..2f34aef0932 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -183,37 +183,29 @@ public: }; -template -class AggregateFunctionNullUnaryBase : public AggregateFunctionNullBase +/** There are two cases: for single argument and variadic. + * Code for single argument is much more efficient. + */ +template +class AggregateFunctionNullUnary final + : public AggregateFunctionNullBase> { public: - AggregateFunctionNullUnaryBase(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) - : AggregateFunctionNullBase(std::move(nested_function_), arguments, params) + AggregateFunctionNullUnary(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) + : AggregateFunctionNullBase>(std::move(nested_function_), arguments, params) { } - virtual bool singleFilter(const IColumn ** columns, size_t row_num) const = 0; - void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override { const ColumnNullable * column = assert_cast(columns[0]); const IColumn * nested_column = &column->getNestedColumn(); - - if constexpr (other_filter) + if (!column->isNullAt(row_num)) { - if (!column->isNullAt(row_num) && singleFilter(columns, row_num)) - { - this->setFlag(place); - this->nested_function->add(this->nestedPlace(place), &nested_column, row_num, arena); - } - } - else - { - if (!column->isNullAt(row_num)) - { - this->setFlag(place); - this->nested_function->add(this->nestedPlace(place), &nested_column, row_num, arena); - } + this->setFlag(place); + this->nested_function->add(this->nestedPlace(place), &nested_column, row_num, arena); } } @@ -223,58 +215,25 @@ public: const IColumn * nested_column = &column->getNestedColumn(); const UInt8 * null_map = column->getNullMapData().data(); + this->nested_function->addBatchSinglePlaceNotNull(batch_size, this->nestedPlace(place), &nested_column, null_map, arena); + if constexpr (result_is_nullable) - { - if constexpr (other_filter) - { - bool flag = false; - for (size_t i = 0; i < batch_size; ++i) - { - if (!null_map[i] && singleFilter(columns, i)) - { - flag = true; - static_cast(this)->add(place, columns, i, arena); - } - } - - if (flag) - this->setFlag(place); - } - else - { - this->nested_function->addBatchSinglePlaceNotNull(batch_size, this->nestedPlace(place), &nested_column, null_map, arena); - - if (!memoryIsByte(null_map, batch_size, 1)) - this->setFlag(place); - } - } + if (!memoryIsByte(null_map, batch_size, 1)) + this->setFlag(place); } }; -/** There are two cases: for single argument and variadic. - * Code for single argument is much more efficient. - */ -template -class AggregateFunctionNullUnary final - : public AggregateFunctionNullUnaryBase> + +template +class AggregateFunctionNullVariadic final + : public AggregateFunctionNullBase> { public: - AggregateFunctionNullUnary(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) - : AggregateFunctionNullUnaryBase>(std::move(nested_function_), arguments, params) - { - } - - bool singleFilter(const IColumn ** /*columns*/, size_t /*row_num*/) const override { return true; } -}; - -template -class AggregateFunctionNullVariadicBase : public AggregateFunctionNullBase -{ -public: - AggregateFunctionNullVariadicBase(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) - : AggregateFunctionNullBase(std::move(nested_function_), arguments, params), - number_of_arguments(arguments.size()) + AggregateFunctionNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) + : AggregateFunctionNullBase>(std::move(nested_function_), arguments, params), + number_of_arguments(arguments.size()) { if (number_of_arguments == 1) throw Exception("Logical error: single argument is passed to AggregateFunctionNullVariadic", ErrorCodes::LOGICAL_ERROR); @@ -287,8 +246,6 @@ public: is_nullable[i] = arguments[i]->isNullable(); } - virtual bool singleFilter(const IColumn ** columns, size_t row_num) const = 0; - void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override { /// This container stores the columns we really pass to the nested function. @@ -311,40 +268,14 @@ public: nested_columns[i] = columns[i]; } - if constexpr (other_filter) - { - if (singleFilter(nested_columns, row_num)) - { - this->setFlag(place); - this->nested_function->add(this->nestedPlace(place), nested_columns, row_num, arena); - } - } - else - { - this->setFlag(place); - this->nested_function->add(this->nestedPlace(place), nested_columns, row_num, arena); - } + this->setFlag(place); + this->nested_function->add(this->nestedPlace(place), nested_columns, row_num, arena); } -protected: +private: enum { MAX_ARGS = 8 }; size_t number_of_arguments = 0; std::array is_nullable; /// Plain array is better than std::vector due to one indirection less. }; -template -class AggregateFunctionNullVariadic final - : public AggregateFunctionNullVariadicBase> -{ -public: - AggregateFunctionNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) - : AggregateFunctionNullVariadicBase>(std::move(nested_function_), arguments, params) - { - } - - bool singleFilter(const IColumn ** /*columns*/, size_t /*row_num*/) const override { return true; } -}; - } From 75af61ea95578aa14969296a5fb9cb32d0f73979 Mon Sep 17 00:00:00 2001 From: Winter Zhang Date: Mon, 24 Aug 2020 18:51:54 +0800 Subject: [PATCH 0082/1911] Try fix style failure --- src/AggregateFunctions/AggregateFunctionIf.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index 8d10fda5f27..47afddaf7ff 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -9,6 +9,7 @@ namespace DB namespace ErrorCodes { + extern const int LOGICAL_ERROR; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } From 4331158d3051437f44c7fa1271e4673272cf8cac Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Aug 2020 16:09:23 +0300 Subject: [PATCH 0083/1911] merge with master --- src/Disks/DiskDecorator.cpp | 15 +++++++++++++++ src/Disks/DiskDecorator.h | 3 +++ .../MergeTree/MergeTreeDataPartWriterInMemory.cpp | 2 +- .../MergeTree/MergeTreeDataPartWriterInMemory.h | 2 +- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 11 ++++++----- 5 files changed, 26 insertions(+), 7 deletions(-) diff --git a/src/Disks/DiskDecorator.cpp b/src/Disks/DiskDecorator.cpp index e55534e347f..7f2ea58d7cf 100644 --- a/src/Disks/DiskDecorator.cpp +++ b/src/Disks/DiskDecorator.cpp @@ -165,4 +165,19 @@ void DiskDecorator::truncateFile(const String & path, size_t size) delegate->truncateFile(path, size); } +int DiskDecorator::open(const String & path, mode_t mode) const +{ + return delegate->open(path, mode); +} + +void DiskDecorator::close(int fd) const +{ + delegate->close(fd); +} + +void DiskDecorator::sync(int fd) const +{ + delegate->sync(fd); +} + } diff --git a/src/Disks/DiskDecorator.h b/src/Disks/DiskDecorator.h index 71bb100c576..f1ddfff4952 100644 --- a/src/Disks/DiskDecorator.h +++ b/src/Disks/DiskDecorator.h @@ -42,6 +42,9 @@ public: void setReadOnly(const String & path) override; void createHardLink(const String & src_path, const String & dst_path) override; void truncateFile(const String & path, size_t size) override; + int open(const String & path, mode_t mode) const override; + void close(int fd) const override; + void sync(int fd) const override; const String getType() const override { return delegate->getType(); } protected: diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp index a7486158737..f0738a1130a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp @@ -70,7 +70,7 @@ void MergeTreeDataPartWriterInMemory::calculateAndSerializePrimaryIndex(const Bl } } -void MergeTreeDataPartWriterInMemory::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) +void MergeTreeDataPartWriterInMemory::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool) { /// If part is empty we still need to initialize block by empty columns. if (!part_in_memory->block) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h index 92e4228a90d..6e59cdd08a9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h @@ -18,7 +18,7 @@ public: void write(const Block & block, const IColumn::Permutation * permutation, const Block & primary_key_block, const Block & skip_indexes_block) override; - void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override; + void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) override; void calculateAndSerializePrimaryIndex(const Block & primary_index_block) override; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index f3a72657be5..b05b970da3b 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -251,6 +251,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa new_data_part->minmax_idx = std::move(minmax_idx); new_data_part->is_temp = true; + std::optional sync_guard; if (new_data_part->isStoredOnDisk()) { /// The name could be non-unique in case of stale files from previous runs. @@ -262,12 +263,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa new_data_part->volume->getDisk()->removeRecursive(full_path); } - const auto disk = new_data_part->volume->getDisk(); - disk->createDirectories(full_path); + const auto disk = new_data_part->volume->getDisk(); + disk->createDirectories(full_path); - std::optional sync_guard; - if (data.getSettings()->fsync_part_directory) - sync_guard.emplace(disk, full_path); + if (data.getSettings()->fsync_part_directory) + sync_guard.emplace(disk, full_path); + } /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) From 1180bad3a73d2d17457c7562f65edee82f7a0af4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 25 Aug 2020 18:02:32 +0300 Subject: [PATCH 0084/1911] Add ability to specify DefaultCompression codec which correspond to settings specified in config.xml --- programs/compressor/Compressor.cpp | 4 +- src/Client/Connection.cpp | 3 +- src/Compression/CompressionCodecDelta.cpp | 5 - src/Compression/CompressionCodecDelta.h | 2 - .../CompressionCodecDoubleDelta.cpp | 7 +- src/Compression/CompressionCodecDoubleDelta.h | 2 - src/Compression/CompressionCodecGorilla.cpp | 7 +- src/Compression/CompressionCodecGorilla.h | 2 - src/Compression/CompressionCodecMultiple.cpp | 29 +--- src/Compression/CompressionCodecMultiple.h | 4 +- src/Compression/CompressionFactory.cpp | 153 +++++++++++++++--- src/Compression/CompressionFactory.h | 35 +++- src/Compression/ICompressionCodec.h | 4 - src/DataStreams/TemporaryFileStream.h | 2 +- src/Interpreters/InterpreterAlterQuery.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 4 +- src/Interpreters/InterpreterDescribeQuery.cpp | 2 +- src/Server/TCPHandler.cpp | 6 +- src/Storages/AlterCommands.cpp | 29 ++-- src/Storages/AlterCommands.h | 4 +- src/Storages/ColumnsDescription.cpp | 8 +- src/Storages/ColumnsDescription.h | 3 +- src/Storages/CompressionCodecSelector.h | 2 +- src/Storages/System/StorageSystemColumns.cpp | 2 +- .../__init__.py | 0 .../configs/default_compression.xml | 29 ++++ .../test_default_compression_codec/test.py | 119 ++++++++++++++ .../01455_default_compression.reference | 8 + .../0_stateless/01455_default_compression.sql | 29 ++++ 29 files changed, 389 insertions(+), 117 deletions(-) create mode 100644 tests/integration/test_default_compression_codec/__init__.py create mode 100644 tests/integration/test_default_compression_codec/configs/default_compression.xml create mode 100644 tests/integration/test_default_compression_codec/test.py create mode 100644 tests/queries/0_stateless/01455_default_compression.reference create mode 100644 tests/queries/0_stateless/01455_default_compression.sql diff --git a/programs/compressor/Compressor.cpp b/programs/compressor/Compressor.cpp index 701e8ea2d1c..dd906bffe7b 100644 --- a/programs/compressor/Compressor.cpp +++ b/programs/compressor/Compressor.cpp @@ -126,10 +126,10 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) std::string codecs_line = boost::algorithm::join(codecs, ","); auto ast = parseQuery(codec_parser, "(" + codecs_line + ")", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - codec = CompressionCodecFactory::instance().get(ast, nullptr, false); + codec = CompressionCodecFactory::instance().get(ast, nullptr); } else - codec = CompressionCodecFactory::instance().get(method_family, level, false); + codec = CompressionCodecFactory::instance().get(method_family, level); ReadBufferFromFileDescriptor rb(STDIN_FILENO); diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index e93e84c5e94..ed27a878b5a 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -376,7 +376,8 @@ void Connection::sendQuery( if (method == "ZSTD") level = settings->network_zstd_compression_level; - compression_codec = CompressionCodecFactory::instance().get(method, level, !settings->allow_suspicious_codecs); + CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs); + compression_codec = CompressionCodecFactory::instance().get(method, level); } else compression_codec = CompressionCodecFactory::instance().getDefaultCodec(); diff --git a/src/Compression/CompressionCodecDelta.cpp b/src/Compression/CompressionCodecDelta.cpp index c12d7753ddf..7ecef677e5a 100644 --- a/src/Compression/CompressionCodecDelta.cpp +++ b/src/Compression/CompressionCodecDelta.cpp @@ -150,11 +150,6 @@ UInt8 getDeltaBytesSize(DataTypePtr column_type) } -void CompressionCodecDelta::useInfoAboutType(const DataTypePtr & data_type) -{ - delta_bytes_size = getDeltaBytesSize(data_type); -} - void registerCodecDelta(CompressionCodecFactory & factory) { UInt8 method_code = UInt8(CompressionMethodByte::Delta); diff --git a/src/Compression/CompressionCodecDelta.h b/src/Compression/CompressionCodecDelta.h index 7bc5d9ceb7a..ee3af359492 100644 --- a/src/Compression/CompressionCodecDelta.h +++ b/src/Compression/CompressionCodecDelta.h @@ -14,8 +14,6 @@ public: String getCodecDesc() const override; - void useInfoAboutType(const DataTypePtr & data_type) override; - protected: UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index bc8a2a4b541..a9a69a50693 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -406,11 +406,6 @@ void CompressionCodecDoubleDelta::doDecompressData(const char * source, UInt32 s } } -void CompressionCodecDoubleDelta::useInfoAboutType(const DataTypePtr & data_type) -{ - data_bytes_size = getDataBytesSize(data_type); -} - void registerCodecDoubleDelta(CompressionCodecFactory & factory) { UInt8 method_code = UInt8(CompressionMethodByte::DoubleDelta); @@ -420,7 +415,7 @@ void registerCodecDoubleDelta(CompressionCodecFactory & factory) if (arguments) throw Exception("Codec DoubleDelta does not accept any arguments", ErrorCodes::BAD_ARGUMENTS); - UInt8 data_bytes_size = column_type ? getDataBytesSize(column_type) : 0; /// Maybe postponed to the call to "useInfoAboutType" + UInt8 data_bytes_size = column_type ? getDataBytesSize(column_type) : 0; return std::make_shared(data_bytes_size); }); } diff --git a/src/Compression/CompressionCodecDoubleDelta.h b/src/Compression/CompressionCodecDoubleDelta.h index d5e0770c663..dc3ec8bc243 100644 --- a/src/Compression/CompressionCodecDoubleDelta.h +++ b/src/Compression/CompressionCodecDoubleDelta.h @@ -100,8 +100,6 @@ public: String getCodecDesc() const override; - void useInfoAboutType(const DataTypePtr & data_type) override; - protected: UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; diff --git a/src/Compression/CompressionCodecGorilla.cpp b/src/Compression/CompressionCodecGorilla.cpp index 4a09d16963f..b4f1cfc3c46 100644 --- a/src/Compression/CompressionCodecGorilla.cpp +++ b/src/Compression/CompressionCodecGorilla.cpp @@ -322,11 +322,6 @@ void CompressionCodecGorilla::doDecompressData(const char * source, UInt32 sourc } } -void CompressionCodecGorilla::useInfoAboutType(const DataTypePtr & data_type) -{ - data_bytes_size = getDataBytesSize(data_type); -} - void registerCodecGorilla(CompressionCodecFactory & factory) { UInt8 method_code = UInt8(CompressionMethodByte::Gorilla); @@ -336,7 +331,7 @@ void registerCodecGorilla(CompressionCodecFactory & factory) if (arguments) throw Exception("Codec Gorilla does not accept any arguments", ErrorCodes::BAD_ARGUMENTS); - UInt8 data_bytes_size = column_type ? getDataBytesSize(column_type) : 0; /// Maybe postponed to the call to "useInfoAboutType" + UInt8 data_bytes_size = column_type ? getDataBytesSize(column_type) : 0; return std::make_shared(data_bytes_size); }); } diff --git a/src/Compression/CompressionCodecGorilla.h b/src/Compression/CompressionCodecGorilla.h index 27a6d278978..48c2bdad1ee 100644 --- a/src/Compression/CompressionCodecGorilla.h +++ b/src/Compression/CompressionCodecGorilla.h @@ -97,8 +97,6 @@ public: String getCodecDesc() const override; - void useInfoAboutType(const DataTypePtr & data_type) override; - protected: UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; diff --git a/src/Compression/CompressionCodecMultiple.cpp b/src/Compression/CompressionCodecMultiple.cpp index 6f4ebca90b8..485d962d592 100644 --- a/src/Compression/CompressionCodecMultiple.cpp +++ b/src/Compression/CompressionCodecMultiple.cpp @@ -19,30 +19,9 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -CompressionCodecMultiple::CompressionCodecMultiple(Codecs codecs_, bool sanity_check) +CompressionCodecMultiple::CompressionCodecMultiple(Codecs codecs_) : codecs(codecs_) { - if (sanity_check) - { - /// It does not make sense to apply any transformations after generic compression algorithm - /// So, generic compression can be only one and only at the end. - bool has_generic_compression = false; - for (const auto & codec : codecs) - { - if (codec->isNone()) - throw Exception("It does not make sense to have codec NONE along with other compression codecs: " + getCodecDescImpl() - + ". (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", - ErrorCodes::BAD_ARGUMENTS); - - if (has_generic_compression) - throw Exception("The combination of compression codecs " + getCodecDescImpl() + " is meaningless," - " because it does not make sense to apply any transformations after generic compression algorithm." - " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS); - - if (codec->isGenericCompression()) - has_generic_compression = true; - } - } } uint8_t CompressionCodecMultiple::getMethodByte() const @@ -103,12 +82,6 @@ UInt32 CompressionCodecMultiple::doCompressData(const char * source, UInt32 sour return 1 + codecs.size() + source_size; } -void CompressionCodecMultiple::useInfoAboutType(const DataTypePtr & data_type) -{ - for (auto & codec : codecs) - codec->useInfoAboutType(data_type); -} - void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 decompressed_size) const { if (source_size < 1 || !source[0]) diff --git a/src/Compression/CompressionCodecMultiple.h b/src/Compression/CompressionCodecMultiple.h index 7737e91ad92..8059b57bacb 100644 --- a/src/Compression/CompressionCodecMultiple.h +++ b/src/Compression/CompressionCodecMultiple.h @@ -9,7 +9,7 @@ class CompressionCodecMultiple final : public ICompressionCodec { public: CompressionCodecMultiple() = default; /// Need for CompressionFactory to register codec by method byte. - CompressionCodecMultiple(Codecs codecs_, bool sanity_check); + CompressionCodecMultiple(Codecs codecs_); uint8_t getMethodByte() const override; @@ -17,8 +17,6 @@ public: UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; - void useInfoAboutType(const DataTypePtr & data_type) override; - protected: UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 1df4c443699..0f1a8229870 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -8,6 +8,7 @@ #include #include +#include namespace DB { @@ -20,62 +21,172 @@ namespace ErrorCodes extern const int DATA_TYPE_CANNOT_HAVE_ARGUMENTS; } +static constexpr auto DEFAULT_CODEC_NAME = "DefaultCompression"; + CompressionCodecPtr CompressionCodecFactory::getDefaultCodec() const { return default_codec; } -CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std::optional level, bool sanity_check) const +CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std::optional level) const { if (level) { auto literal = std::make_shared(static_cast(*level)); - return get(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), {}, sanity_check); + return get(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), {}); } else { auto identifier = std::make_shared(Poco::toUpper(family_name)); - return get(makeASTFunction("CODEC", identifier), {}, sanity_check); + return get(makeASTFunction("CODEC", identifier), {}); } } - -CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, DataTypePtr column_type, bool sanity_check) const +void CompressionCodecFactory::validateCodec(const String & family_name, std::optional level, bool sanity_check) const { + if (level) + { + auto literal = std::make_shared(static_cast(*level)); + validateCodecAndGetDescription(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), {}, sanity_check); + } + else + { + auto identifier = std::make_shared(Poco::toUpper(family_name)); + validateCodecAndGetDescription(makeASTFunction("CODEC", identifier), {}, sanity_check); + } +} + +CompressionCodecDescription CompressionCodecFactory::validateCodecAndGetDescription(const ASTPtr & ast, DataTypePtr column_type, bool sanity_check) const +{ + if (const auto * func = ast->as()) + { + Strings codecs_descriptions; + + bool is_compression = false; + bool has_none = false; + std::optional generic_compression_codec_pos; + + for (size_t i = 0; i < func->arguments->children.size(); ++i) + { + const auto & inner_codec_ast = func->arguments->children[i]; + String codec_family_name; + ASTPtr codec_arguments; + if (const auto * family_name = inner_codec_ast->as()) + { + codec_family_name = family_name->name; + codec_arguments = {}; + } + else if (const auto * ast_func = inner_codec_ast->as()) + { + codec_family_name = ast_func->name; + codec_arguments = ast_func->arguments; + } + else + throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE); + + /// Default codec replaced with current default codec which may dependend on different + /// settings (and properties of data) in runtime. + CompressionCodecPtr result_codec; + if (codec_family_name == DEFAULT_CODEC_NAME) + { + if (codec_arguments != nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "{} codec cannot have any arguments, it's just an alias for codec specified in config.xml", DEFAULT_CODEC_NAME); + + result_codec = default_codec; + codecs_descriptions.emplace_back(codec_family_name); + } + else + { + result_codec = getImpl(codec_family_name, codec_arguments, column_type); + codecs_descriptions.emplace_back(result_codec->getCodecDesc()); + } + + is_compression |= result_codec->isCompression(); + has_none |= result_codec->isNone(); + + if (!generic_compression_codec_pos && result_codec->isGenericCompression()) + generic_compression_codec_pos = i; + } + + String codec_description = boost::algorithm::join(codecs_descriptions, ", "); + + if (sanity_check) + { + if (codecs_descriptions.size() > 1 && has_none) + throw Exception( + "It does not make sense to have codec NONE along with other compression codecs: " + codec_description + + ". (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", + ErrorCodes::BAD_ARGUMENTS); + + /// Allow to explicitly specify single NONE codec if user don't want any compression. + /// But applying other transformations solely without compression (e.g. Delta) does not make sense. + if (!is_compression && !has_none) + throw Exception( + "Compression codec " + codec_description + + " does not compress anything." + " You may want to add generic compression algorithm after other transformations, like: " + + codec_description + + ", LZ4." + " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", + ErrorCodes::BAD_ARGUMENTS); + + /// It does not make sense to apply any transformations after generic compression algorithm + /// So, generic compression can be only one and only at the end. + if (generic_compression_codec_pos && *generic_compression_codec_pos != codecs_descriptions.size() - 1) + throw Exception("The combination of compression codecs " + codec_description + " is meaningless," + " because it does not make sense to apply any transformations after generic compression algorithm." + " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS); + + } + return CompressionCodecDescription{ast, codec_description}; + } + + throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC); +} + +CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, DataTypePtr column_type, CompressionCodecPtr current_default) const +{ + if (current_default == nullptr) + current_default = default_codec; + if (const auto * func = ast->as()) { Codecs codecs; codecs.reserve(func->arguments->children.size()); for (const auto & inner_codec_ast : func->arguments->children) { + String codec_family_name; + ASTPtr codec_arguments; if (const auto * family_name = inner_codec_ast->as()) - codecs.emplace_back(getImpl(family_name->name, {}, column_type)); + { + codec_family_name = family_name->name; + codec_arguments = {}; + } else if (const auto * ast_func = inner_codec_ast->as()) - codecs.emplace_back(getImpl(ast_func->name, ast_func->arguments, column_type)); + { + codec_family_name = ast_func->name; + codec_arguments = ast_func->arguments; + } else throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE); + + if (codec_family_name == DEFAULT_CODEC_NAME) + codecs.emplace_back(current_default); + else + codecs.emplace_back(getImpl(codec_family_name, codec_arguments, column_type)); } CompressionCodecPtr res; if (codecs.size() == 1) - res = codecs.back(); + return codecs.back(); else if (codecs.size() > 1) - res = std::make_shared(codecs, sanity_check); - - /// Allow to explicitly specify single NONE codec if user don't want any compression. - /// But applying other transformations solely without compression (e.g. Delta) does not make sense. - if (sanity_check && !res->isCompression() && !res->isNone()) - throw Exception("Compression codec " + res->getCodecDesc() + " does not compress anything." - " You may want to add generic compression algorithm after other transformations, like: " + res->getCodecDesc() + ", LZ4." - " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", - ErrorCodes::BAD_ARGUMENTS); - - return res; + return std::make_shared(codecs); } - throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC); + throw Exception("Unexpected AST structure for compression codec: " + queryToString(ast), ErrorCodes::UNEXPECTED_AST_STRUCTURE); } CompressionCodecPtr CompressionCodecFactory::get(const uint8_t byte_code) const @@ -163,7 +274,7 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecGorilla(*this); registerCodecMultiple(*this); - default_codec = get("LZ4", {}, false); + default_codec = get("LZ4", {}); } CompressionCodecFactory & CompressionCodecFactory::instance() diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index 58799538f8c..de5672365a9 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -20,6 +20,23 @@ using CompressionCodecPtr = std::shared_ptr; using CodecNameWithLevel = std::pair>; +/// Description for codec. AST and description may differ, because some codecs +/// may add some default parameters to their description. +/// +/// NOTE: We store both AST and preprocessed description. It would be simplier to use only AST +/// everywhere, but historically we store preprocessed codecs description in metadata files, +/// so now we cannot replace it with AST specified by user. +struct CompressionCodecDescription +{ + /// AST specified by user, for example "CODEC(Delta, LZ4HC)". Used to get + /// ICompressionCodec object from factory. + ASTPtr ast; + + /// String description with substituted codecs parameters: "Delta(4), LZ4HC(5)". + /// Stored in text representation of ColumnsDescription. + String description; +}; + /** Creates a codec object by name of compression algorithm family and parameters. */ class CompressionCodecFactory final : private boost::noncopyable @@ -37,16 +54,24 @@ public: /// Return default codec (currently LZ4) CompressionCodecPtr getDefaultCodec() const; - /// Get codec by AST and possible column_type - /// some codecs can use information about type to improve inner settings - /// but every codec should be able to work without information about type - CompressionCodecPtr get(const ASTPtr & ast, DataTypePtr column_type, bool sanity_check) const; + /// Validate codecs AST specified by user and parses codecs description (substitute default parameters) + CompressionCodecDescription validateCodecAndGetDescription(const ASTPtr & ast, DataTypePtr column_type, bool sanity_check) const; + + /// Validate codecs AST specified by user + void validateCodec(const String & family_name, std::optional level, bool sanity_check) const; + + /// Get codec by AST and possible column_type. Some codecs can use + /// information about type to improve inner settings, but every codec should + /// be able to work without information about type. Also AST can contain + /// codec, which can be alias to current default codec, which can be changed + /// in runtime. + CompressionCodecPtr get(const ASTPtr & ast, DataTypePtr column_type, CompressionCodecPtr current_default = nullptr) const; /// Get codec by method byte (no params available) CompressionCodecPtr get(const uint8_t byte_code) const; /// For backward compatibility with config settings - CompressionCodecPtr get(const String & family_name, std::optional level, bool sanity_check) const; + CompressionCodecPtr get(const String & family_name, std::optional level) const; /// Register codec with parameters and column type void registerCompressionCodecWithType(const String & family_name, std::optional byte_code, CreatorWithType creator); diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index dab2f817616..664b88d7ac6 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -17,7 +17,6 @@ using Codecs = std::vector; class IDataType; using DataTypePtr = std::shared_ptr; - /** * Represents interface for compression codecs like LZ4, ZSTD, etc. */ @@ -59,9 +58,6 @@ public: /// Read method byte from compressed source static uint8_t readMethod(const char * source); - /// Some codecs may use information about column type which appears after codec creation - virtual void useInfoAboutType(const DataTypePtr & /* data_type */) {} - /// Return true if this codec actually compressing something. Otherwise it can be just transformation that helps compression (e.g. Delta). virtual bool isCompression() const = 0; diff --git a/src/DataStreams/TemporaryFileStream.h b/src/DataStreams/TemporaryFileStream.h index ddb2190f175..6871800a540 100644 --- a/src/DataStreams/TemporaryFileStream.h +++ b/src/DataStreams/TemporaryFileStream.h @@ -37,7 +37,7 @@ struct TemporaryFileStream std::atomic * is_cancelled, const std::string & codec) { WriteBufferFromFile file_buf(path); - CompressedWriteBuffer compressed_buf(file_buf, CompressionCodecFactory::instance().get(codec, {}, false)); + CompressedWriteBuffer compressed_buf(file_buf, CompressionCodecFactory::instance().get(codec, {})); NativeBlockOutputStream output(compressed_buf, 0, header); copyData(input, output, is_cancelled); } diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index c0ed5d8a36c..8456e292147 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -58,7 +58,7 @@ BlockIO InterpreterAlterQuery::execute() LiveViewCommands live_view_commands; for (ASTAlterCommand * command_ast : alter.command_list->commands) { - if (auto alter_command = AlterCommand::parse(command_ast, !context.getSettingsRef().allow_suspicious_codecs)) + if (auto alter_command = AlterCommand::parse(command_ast)) alter_commands.emplace_back(std::move(*alter_command)); else if (auto partition_command = PartitionCommand::parse(command_ast)) { diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ef81eb41672..8a33275ba15 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -282,7 +282,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) if (column.codec) { - String codec_desc = column.codec->getCodecDesc(); + String codec_desc = column.codec->description; codec_desc = "CODEC(" + codec_desc + ")"; const char * codec_desc_pos = codec_desc.data(); const char * codec_desc_end = codec_desc_pos + codec_desc.size(); @@ -422,7 +422,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( column.comment = col_decl.comment->as().value.get(); if (col_decl.codec) - column.codec = CompressionCodecFactory::instance().get(col_decl.codec, column.type, sanity_check_compression_codecs); + column.codec = CompressionCodecFactory::instance().validateCodecAndGetDescription(col_decl.codec, column.type, sanity_check_compression_codecs); if (col_decl.ttl) column.ttl = col_decl.ttl; diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 94fa748ea15..48da2e33773 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -116,7 +116,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() res_columns[4]->insert(column.comment); if (column.codec) - res_columns[5]->insert(column.codec->getCodecDesc()); + res_columns[5]->insert(column.codec->description); else res_columns[5]->insertDefault(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index ab4ce820666..e4dcfce9102 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1045,8 +1045,12 @@ void TCPHandler::initBlockOutput(const Block & block) level = query_settings.network_zstd_compression_level; if (state.compression == Protocol::Compression::Enable) + { + CompressionCodecFactory::instance().validateCodec(method, level, !query_settings.allow_suspicious_codecs); + state.maybe_compressed_out = std::make_shared( - *out, CompressionCodecFactory::instance().get(method, level, !query_settings.allow_suspicious_codecs)); + *out, CompressionCodecFactory::instance().get(method, level)); + } else state.maybe_compressed_out = out; } diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 03e8c8efd63..497ee4feb7d 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -44,10 +44,9 @@ namespace ErrorCodes } -std::optional AlterCommand::parse(const ASTAlterCommand * command_ast, bool sanity_check_compression_codecs) +std::optional AlterCommand::parse(const ASTAlterCommand * command_ast) { const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); - const CompressionCodecFactory & compression_codec_factory = CompressionCodecFactory::instance(); if (command_ast->type == ASTAlterCommand::ADD_COLUMN) { @@ -75,7 +74,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ } if (ast_col_decl.codec) - command.codec = compression_codec_factory.get(ast_col_decl.codec, command.data_type, sanity_check_compression_codecs); + command.codec = ast_col_decl.codec; if (command_ast->column) command.after_column = getIdentifierName(command_ast->column); @@ -132,7 +131,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.ttl = ast_col_decl.ttl; if (ast_col_decl.codec) - command.codec = compression_codec_factory.get(ast_col_decl.codec, command.data_type, sanity_check_compression_codecs); + command.codec = ast_col_decl.codec; if (command_ast->column) command.after_column = getIdentifierName(command_ast->column); @@ -271,7 +270,9 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con if (comment) column.comment = *comment; - column.codec = codec; + if (codec) + column.codec = CompressionCodecFactory::instance().validateCodecAndGetDescription(codec, data_type, false); + column.ttl = ttl; metadata.columns.add(column, after_column, first); @@ -290,16 +291,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con metadata.columns.modify(column_name, after_column, first, [&](ColumnDescription & column) { if (codec) - { - /// User doesn't specify data type, it means that datatype doesn't change - /// let's use info about old type - if (data_type == nullptr) - codec->useInfoAboutType(column.type); - else /// use info about new DataType - codec->useInfoAboutType(data_type); - - column.codec = codec; - } + column.codec = CompressionCodecFactory::instance().validateCodecAndGetDescription(codec, data_type ? data_type : column.type, false); if (comment) column.comment = *comment; @@ -822,6 +814,9 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con throw Exception{"Data type have to be specified for column " + backQuote(column_name) + " to add", ErrorCodes::BAD_ARGUMENTS}; + if (command.codec) + CompressionCodecFactory::instance().validateCodecAndGetDescription(command.codec, command.data_type, !context.getSettingsRef().allow_suspicious_codecs); + all_columns.add(ColumnDescription(column_name, command.data_type)); } else if (command.type == AlterCommand::MODIFY_COLUMN) @@ -838,6 +833,10 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con if (renamed_columns.count(column_name)) throw Exception{"Cannot rename and modify the same column " + backQuote(column_name) + " in a single ALTER query", ErrorCodes::NOT_IMPLEMENTED}; + + if (command.codec) + CompressionCodecFactory::instance().validateCodecAndGetDescription(command.codec, command.data_type, !context.getSettingsRef().allow_suspicious_codecs); + modified_columns.emplace(column_name); } else if (command.type == AlterCommand::DROP_COLUMN) diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index 69b6bdff44d..47198993c90 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -92,7 +92,7 @@ struct AlterCommand bool clear = false; /// For ADD and MODIFY - CompressionCodecPtr codec = nullptr; + ASTPtr codec = nullptr; /// For MODIFY SETTING SettingsChanges settings_changes; @@ -103,7 +103,7 @@ struct AlterCommand /// Target column name String rename_to; - static std::optional parse(const ASTAlterCommand * command, bool sanity_check_compression_codecs); + static std::optional parse(const ASTAlterCommand * command); void apply(StorageInMemoryMetadata & metadata, const Context & context) const; diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 869cc5cdcdb..41f88c5c2ae 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -49,7 +49,7 @@ ColumnDescription::ColumnDescription(String name_, DataTypePtr type_) bool ColumnDescription::operator==(const ColumnDescription & other) const { - auto codec_str = [](const CompressionCodecPtr & codec_ptr) { return codec_ptr ? codec_ptr->getCodecDesc() : String(); }; + auto codec_str = [](const auto & codec_desc) { return codec_desc ? codec_desc->description : String{}; }; auto ttl_str = [](const ASTPtr & ttl_ast) { return ttl_ast ? queryToString(ttl_ast) : String{}; }; return name == other.name @@ -85,7 +85,7 @@ void ColumnDescription::writeText(WriteBuffer & buf) const { writeChar('\t', buf); DB::writeText("CODEC(", buf); - DB::writeText(codec->getCodecDesc(), buf); + DB::writeText(codec->description, buf); DB::writeText(")", buf); } @@ -120,7 +120,7 @@ void ColumnDescription::readText(ReadBuffer & buf) comment = col_ast->comment->as().value.get(); if (col_ast->codec) - codec = CompressionCodecFactory::instance().get(col_ast->codec, type, false); + codec = CompressionCodecFactory::instance().validateCodecAndGetDescription(col_ast->codec, type, false); if (col_ast->ttl) ttl = col_ast->ttl; @@ -414,7 +414,7 @@ CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_ if (it == columns.get<1>().end() || !it->codec) return default_codec; - return it->codec; + return CompressionCodecFactory::instance().get(it->codec->ast, it->type, default_codec); } CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_name) const diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index a95705d79f4..c9b4f6e38bd 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -30,7 +31,7 @@ struct ColumnDescription DataTypePtr type; ColumnDefault default_desc; String comment; - CompressionCodecPtr codec; + std::optional codec; ASTPtr ttl; ColumnDescription() = default; diff --git a/src/Storages/CompressionCodecSelector.h b/src/Storages/CompressionCodecSelector.h index 0924550fcea..d194adc9317 100644 --- a/src/Storages/CompressionCodecSelector.h +++ b/src/Storages/CompressionCodecSelector.h @@ -91,7 +91,7 @@ public: for (const auto & element : elements) if (element.check(part_size, part_size_ratio)) - res = factory.get(element.family_name, element.level, false); + res = factory.get(element.family_name, element.level); return res; } diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index d737eff3062..d714897a684 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -214,7 +214,7 @@ protected: if (columns_mask[src_index++]) { if (column.codec) - res_columns[res_index++]->insert("CODEC(" + column.codec->getCodecDesc() + ")"); + res_columns[res_index++]->insert("CODEC(" + column.codec->description + ")"); else res_columns[res_index++]->insertDefault(); } diff --git a/tests/integration/test_default_compression_codec/__init__.py b/tests/integration/test_default_compression_codec/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_default_compression_codec/configs/default_compression.xml b/tests/integration/test_default_compression_codec/configs/default_compression.xml new file mode 100644 index 00000000000..bc9d7ad8e20 --- /dev/null +++ b/tests/integration/test_default_compression_codec/configs/default_compression.xml @@ -0,0 +1,29 @@ + + + + + 0 + 0 + + zstd + 10 + + + + + 1024 + 0 + + lz4hc + 10 + + + + 11024 + 0 + + lz4 + + + + diff --git a/tests/integration/test_default_compression_codec/test.py b/tests/integration/test_default_compression_codec/test.py new file mode 100644 index 00000000000..b2e18051273 --- /dev/null +++ b/tests/integration/test_default_compression_codec/test.py @@ -0,0 +1,119 @@ +import string +import random +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance('node1', main_configs=['configs/default_compression.xml'], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=['configs/default_compression.xml'], with_zookeeper=True) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def get_compression_codec_byte(node, table_name, part_name): + cmd = "tail -c +17 /var/lib/clickhouse/data/default/{}/{}/data1.bin | od -x -N 1 | head -n 1 | awk '{{print $2}}'".format(table_name, part_name) + return node.exec_in_container(["bash", "-c", cmd]).strip() + + +def get_second_multiple_codec_byte(node, table_name, part_name): + cmd = "tail -c +17 /var/lib/clickhouse/data/default/{}/{}/data1.bin | od -x -j 11 -N 1 | head -n 1 | awk '{{print $2}}'".format(table_name, part_name) + return node.exec_in_container(["bash", "-c", cmd]).strip() + + +def get_random_string(length): + return ''.join(random.choice(string.ascii_uppercase + string.digits) for _ in range(length)) + + +CODECS_MAPPING = { + 'LZ4': '0082', + 'LZ4HC': '0082', # not an error, same byte + 'ZSTD': '0090', + 'Multiple': '0091', +} + + +def test_default_codec_single(start_cluster): + for i, node in enumerate([node1, node2]): + node.query(""" + CREATE TABLE compression_table ( + key UInt64, + data1 String CODEC(DefaultCompression) + ) ENGINE = ReplicatedMergeTree('/t', '{}') ORDER BY tuple() PARTITION BY key; + """.format(i)) + + # ZSTD(10) and ZSTD(10) after merge + node1.query("INSERT INTO compression_table VALUES (1, 'x')") + + # ZSTD(10) and LZ4HC(10) after merge + node1.query("INSERT INTO compression_table VALUES (2, '{}')".format(get_random_string(2048))) + + # ZSTD(10) and LZ4 after merge + node1.query("INSERT INTO compression_table VALUES (3, '{}')".format(get_random_string(12048))) + + # Same codec for all + assert get_compression_codec_byte(node1, "compression_table", "1_0_0_0") == CODECS_MAPPING['ZSTD'] + assert get_compression_codec_byte(node1, "compression_table", "2_0_0_0") == CODECS_MAPPING['ZSTD'] + assert get_compression_codec_byte(node1, "compression_table", "3_0_0_0") == CODECS_MAPPING['ZSTD'] + + # just to be sure that replication works + node2.query("SYSTEM SYNC REPLICA compression_table", timeout=15) + + node1.query("OPTIMIZE TABLE compression_table FINAL") + + assert get_compression_codec_byte(node1, "compression_table", "1_0_0_1") == CODECS_MAPPING['ZSTD'] + assert get_compression_codec_byte(node1, "compression_table", "2_0_0_1") == CODECS_MAPPING['LZ4HC'] + assert get_compression_codec_byte(node1, "compression_table", "3_0_0_1") == CODECS_MAPPING['LZ4'] + + assert node1.query("SELECT COUNT() FROM compression_table") == "3\n" + assert node2.query("SELECT COUNT() FROM compression_table") == "3\n" + + +def test_default_codec_multiple(start_cluster): + for i, node in enumerate([node1, node2]): + node.query(""" + CREATE TABLE compression_table_multiple ( + key UInt64, + data1 String CODEC(NONE, DefaultCompression) + ) ENGINE = ReplicatedMergeTree('/d', '{}') ORDER BY tuple() PARTITION BY key; + """.format(i), settings={"allow_suspicious_codecs": 1}) + + # ZSTD(10) and ZSTD(10) after merge + node1.query("INSERT INTO compression_table_multiple VALUES (1, 'x')") + + # ZSTD(10) and LZ4HC(10) after merge + node1.query("INSERT INTO compression_table_multiple VALUES (2, '{}')".format(get_random_string(2048))) + + # ZSTD(10) and LZ4 after merge + node1.query("INSERT INTO compression_table_multiple VALUES (3, '{}')".format(get_random_string(12048))) + + # Same codec for all + assert get_compression_codec_byte(node1, "compression_table_multiple", "1_0_0_0") == CODECS_MAPPING['Multiple'] + assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "1_0_0_0") == CODECS_MAPPING['ZSTD'] + assert get_compression_codec_byte(node1, "compression_table_multiple", "2_0_0_0") == CODECS_MAPPING['Multiple'] + assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "2_0_0_0") == CODECS_MAPPING['ZSTD'] + assert get_compression_codec_byte(node1, "compression_table_multiple", "3_0_0_0") == CODECS_MAPPING['Multiple'] + assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "3_0_0_0") == CODECS_MAPPING['ZSTD'] + + node2.query("SYSTEM SYNC REPLICA compression_table_multiple", timeout=15) + + node1.query("OPTIMIZE TABLE compression_table_multiple FINAL") + + assert get_compression_codec_byte(node1, "compression_table_multiple", "1_0_0_1") == CODECS_MAPPING['Multiple'] + assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "1_0_0_1") == CODECS_MAPPING['ZSTD'] + assert get_compression_codec_byte(node1, "compression_table_multiple", "2_0_0_1") == CODECS_MAPPING['Multiple'] + assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "2_0_0_1") == CODECS_MAPPING['LZ4HC'] + assert get_compression_codec_byte(node1, "compression_table_multiple", "3_0_0_1") == CODECS_MAPPING['Multiple'] + assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "3_0_0_1") == CODECS_MAPPING['LZ4'] + + assert node1.query("SELECT COUNT() FROM compression_table_multiple") == "3\n" + assert node2.query("SELECT COUNT() FROM compression_table_multiple") == "3\n" diff --git a/tests/queries/0_stateless/01455_default_compression.reference b/tests/queries/0_stateless/01455_default_compression.reference new file mode 100644 index 00000000000..33e5355fa48 --- /dev/null +++ b/tests/queries/0_stateless/01455_default_compression.reference @@ -0,0 +1,8 @@ +1 1 1 1 +1 1 1 1 +2 2 2 2 +key UInt64 +value1 String DefaultCompression +value2 UInt64 Delta(8), DefaultCompression +value3 String DefaultCompression +CREATE TABLE default.compress_table\n(\n `key` UInt64,\n `value1` String CODEC(DefaultCompression),\n `value2` UInt64 CODEC(Delta(8), DefaultCompression),\n `value3` String CODEC(DefaultCompression)\n)\nENGINE = MergeTree()\nORDER BY key\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01455_default_compression.sql b/tests/queries/0_stateless/01455_default_compression.sql new file mode 100644 index 00000000000..96939b798da --- /dev/null +++ b/tests/queries/0_stateless/01455_default_compression.sql @@ -0,0 +1,29 @@ +DROP TABLE IF EXISTS compress_table; + +CREATE TABLE compress_table +( + key UInt64, + value1 String CODEC(DefaultCompression), + value2 UInt64 CODEC(Delta, DefaultCompression), + value3 String CODEC(ZSTD(10)) +) +ENGINE = MergeTree() +ORDER BY key; + +INSERT INTO compress_table VALUES(1, '1', '1', '1'); + +SELECT * FROM compress_table; + +ALTER TABLE compress_table MODIFY COLUMN value3 CODEC(DefaultCompression); + +INSERT INTO compress_table VALUES(2, '2', '2', '2'); + +SELECT * FROM compress_table ORDER BY key; + +DESCRIBE TABLE compress_table; + +SHOW CREATE TABLE compress_table; + +ALTER TABLE compress_table MODIFY COLUMN value2 CODEC(DefaultCompression(5)); --{serverError 36} + +DROP TABLE IF EXISTS compress_table; From 7bf0b742934f5e0530aa3752f61b0caa629cd60d Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 25 Aug 2020 18:22:21 +0300 Subject: [PATCH 0085/1911] Fix style --- src/Compression/CompressionCodecMultiple.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Compression/CompressionCodecMultiple.cpp b/src/Compression/CompressionCodecMultiple.cpp index 485d962d592..4b9a354d9a4 100644 --- a/src/Compression/CompressionCodecMultiple.cpp +++ b/src/Compression/CompressionCodecMultiple.cpp @@ -16,7 +16,6 @@ namespace DB namespace ErrorCodes { extern const int CORRUPTED_DATA; - extern const int BAD_ARGUMENTS; } CompressionCodecMultiple::CompressionCodecMultiple(Codecs codecs_) From 9ca8c5e77a8d73616a45a8289d6f4830bc3eff51 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 25 Aug 2020 18:53:06 +0300 Subject: [PATCH 0086/1911] Rename DefaultCompression to simple Default --- src/Compression/CompressionFactory.cpp | 2 +- tests/integration/test_default_compression_codec/test.py | 4 ++-- .../0_stateless/01455_default_compression.reference | 8 ++++---- tests/queries/0_stateless/01455_default_compression.sql | 8 ++++---- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 0f1a8229870..8c817b6bb37 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes extern const int DATA_TYPE_CANNOT_HAVE_ARGUMENTS; } -static constexpr auto DEFAULT_CODEC_NAME = "DefaultCompression"; +static constexpr auto DEFAULT_CODEC_NAME = "Default"; CompressionCodecPtr CompressionCodecFactory::getDefaultCodec() const { diff --git a/tests/integration/test_default_compression_codec/test.py b/tests/integration/test_default_compression_codec/test.py index b2e18051273..4de4aba63f6 100644 --- a/tests/integration/test_default_compression_codec/test.py +++ b/tests/integration/test_default_compression_codec/test.py @@ -47,7 +47,7 @@ def test_default_codec_single(start_cluster): node.query(""" CREATE TABLE compression_table ( key UInt64, - data1 String CODEC(DefaultCompression) + data1 String CODEC(Default) ) ENGINE = ReplicatedMergeTree('/t', '{}') ORDER BY tuple() PARTITION BY key; """.format(i)) @@ -83,7 +83,7 @@ def test_default_codec_multiple(start_cluster): node.query(""" CREATE TABLE compression_table_multiple ( key UInt64, - data1 String CODEC(NONE, DefaultCompression) + data1 String CODEC(NONE, Default) ) ENGINE = ReplicatedMergeTree('/d', '{}') ORDER BY tuple() PARTITION BY key; """.format(i), settings={"allow_suspicious_codecs": 1}) diff --git a/tests/queries/0_stateless/01455_default_compression.reference b/tests/queries/0_stateless/01455_default_compression.reference index 33e5355fa48..711ef2d48d2 100644 --- a/tests/queries/0_stateless/01455_default_compression.reference +++ b/tests/queries/0_stateless/01455_default_compression.reference @@ -2,7 +2,7 @@ 1 1 1 1 2 2 2 2 key UInt64 -value1 String DefaultCompression -value2 UInt64 Delta(8), DefaultCompression -value3 String DefaultCompression -CREATE TABLE default.compress_table\n(\n `key` UInt64,\n `value1` String CODEC(DefaultCompression),\n `value2` UInt64 CODEC(Delta(8), DefaultCompression),\n `value3` String CODEC(DefaultCompression)\n)\nENGINE = MergeTree()\nORDER BY key\nSETTINGS index_granularity = 8192 +value1 String Default +value2 UInt64 Delta(8), Default +value3 String Default +CREATE TABLE default.compress_table\n(\n `key` UInt64,\n `value1` String CODEC(Default),\n `value2` UInt64 CODEC(Delta(8), Default),\n `value3` String CODEC(Default)\n)\nENGINE = MergeTree()\nORDER BY key\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01455_default_compression.sql b/tests/queries/0_stateless/01455_default_compression.sql index 96939b798da..5d035197702 100644 --- a/tests/queries/0_stateless/01455_default_compression.sql +++ b/tests/queries/0_stateless/01455_default_compression.sql @@ -3,8 +3,8 @@ DROP TABLE IF EXISTS compress_table; CREATE TABLE compress_table ( key UInt64, - value1 String CODEC(DefaultCompression), - value2 UInt64 CODEC(Delta, DefaultCompression), + value1 String CODEC(Default), + value2 UInt64 CODEC(Delta, Default), value3 String CODEC(ZSTD(10)) ) ENGINE = MergeTree() @@ -14,7 +14,7 @@ INSERT INTO compress_table VALUES(1, '1', '1', '1'); SELECT * FROM compress_table; -ALTER TABLE compress_table MODIFY COLUMN value3 CODEC(DefaultCompression); +ALTER TABLE compress_table MODIFY COLUMN value3 CODEC(Default); INSERT INTO compress_table VALUES(2, '2', '2', '2'); @@ -24,6 +24,6 @@ DESCRIBE TABLE compress_table; SHOW CREATE TABLE compress_table; -ALTER TABLE compress_table MODIFY COLUMN value2 CODEC(DefaultCompression(5)); --{serverError 36} +ALTER TABLE compress_table MODIFY COLUMN value2 CODEC(Default(5)); --{serverError 36} DROP TABLE IF EXISTS compress_table; From 308e094d04401144603fb12a64b4604bb0bde02d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 25 Aug 2020 21:06:21 +0300 Subject: [PATCH 0087/1911] Fix arrayJoin() capturing in lambda Fixes the following LOGICAL_ERROR: $ clickhouse-client -q 'select arrayFilter((a) -> ((a, arrayJoin([[]])) IN (Null, [Null])), [])' 2020.08.16 00:32:01.967102 [ 1744189 ] {b40a5ebd-d710-4f03-bb18-57db67de1181} : Logical error: 'Lambda captured argument arrayJoin(array(array())) not found in required columns.'. clickhouse-server: ../src/Common/Exception.cpp:45: DB::Exception::Exception(const string&, int): Assertion `false' failed. Since there are multiple input columns for arrayJoin(): (gdb) p captured_names_ $6 = std::vector of length 3, capacity 4 = {"arrayJoin(array(array()))", "arrayJoin(array(array()))", "__set"} While FunctionCaptureOverloadResolver cannot handle non-unique columns. --- src/Interpreters/ActionsVisitor.cpp | 15 ++++++++++++++- src/Interpreters/ActionsVisitor.h | 7 +++++++ .../0_stateless/01407_lambda_arrayJoin.reference | 1 + .../0_stateless/01407_lambda_arrayJoin.sql | 6 ++++++ 4 files changed, 28 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01407_lambda_arrayJoin.reference create mode 100644 tests/queries/0_stateless/01407_lambda_arrayJoin.sql diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index f2a1d570773..0df83f11c1f 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -447,6 +447,19 @@ void ScopeStack::addAction(const ExpressionAction & action) } } +void ScopeStack::addActionNoInput(const ExpressionAction & action) +{ + size_t level = 0; + Names required = action.getNeededColumns(); + for (const auto & elem : required) + level = std::max(level, getColumnLevel(elem)); + + Names added; + stack[level].actions->add(action, added); + + stack[level].new_columns.insert(added.begin(), added.end()); +} + ExpressionActionsPtr ScopeStack::popLevel() { ExpressionActionsPtr res = stack.back().actions; @@ -549,7 +562,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & /// It could have been possible to implement arrayJoin which keeps source column, /// but in this case it will always be replicated (as many arrays), which is expensive. String tmp_name = data.getUniqueName("_array_join_" + arg->getColumnName()); - data.addAction(ExpressionAction::copyColumn(arg->getColumnName(), tmp_name)); + data.addActionNoInput(ExpressionAction::copyColumn(arg->getColumnName(), tmp_name)); data.addAction(ExpressionAction::arrayJoin(tmp_name, result_name)); } diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index dbcc54c01d6..d8d85f1c0bf 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -12,6 +12,7 @@ namespace DB class Context; class ASTFunction; +struct ExpressionAction; class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; @@ -49,6 +50,8 @@ struct ScopeStack size_t getColumnLevel(const std::string & name); void addAction(const ExpressionAction & action); + /// For arrayJoin() to avoid double columns in the input. + void addActionNoInput(const ExpressionAction & action); ExpressionActionsPtr popLevel(); @@ -115,6 +118,10 @@ public: { actions_stack.addAction(action); } + void addActionNoInput(const ExpressionAction & action) + { + actions_stack.addActionNoInput(action); + } const Block & getSampleBlock() const { diff --git a/tests/queries/0_stateless/01407_lambda_arrayJoin.reference b/tests/queries/0_stateless/01407_lambda_arrayJoin.reference new file mode 100644 index 00000000000..fe51488c706 --- /dev/null +++ b/tests/queries/0_stateless/01407_lambda_arrayJoin.reference @@ -0,0 +1 @@ +[] diff --git a/tests/queries/0_stateless/01407_lambda_arrayJoin.sql b/tests/queries/0_stateless/01407_lambda_arrayJoin.sql new file mode 100644 index 00000000000..4f34bb59527 --- /dev/null +++ b/tests/queries/0_stateless/01407_lambda_arrayJoin.sql @@ -0,0 +1,6 @@ +SELECT arrayFilter((a) -> ((a, arrayJoin([])) IN (Null, [Null])), []); +SELECT arrayFilter((a) -> ((a, arrayJoin([[]])) IN (Null, [Null])), []); + +-- simplified from the https://clickhouse-test-reports.s3.yandex.net/10373/6c4748a63e7acde2cc3283d96ffec590aae1e724/fuzzer/fuzzer.log#fail1 +SELECT * FROM system.one ARRAY JOIN arrayFilter((a) -> ((a, arrayJoin([])) IN (NULL)), []) AS arr_x; -- { serverError 43; } +SELECT * FROM numbers(1) LEFT ARRAY JOIN arrayFilter((x_0, x_1) -> (arrayJoin([]) IN (NULL)), [], []) AS arr_x; From e5bc5ea419d49c807dada50b1e33966537fd0ada Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 25 Aug 2020 22:30:52 +0300 Subject: [PATCH 0088/1911] Fix compression codec --- src/Compression/tests/gtest_compressionCodec.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/tests/gtest_compressionCodec.cpp b/src/Compression/tests/gtest_compressionCodec.cpp index 6a13b46ac6a..4677efce5da 100644 --- a/src/Compression/tests/gtest_compressionCodec.cpp +++ b/src/Compression/tests/gtest_compressionCodec.cpp @@ -468,7 +468,7 @@ CompressionCodecPtr makeCodec(const std::string & codec_string, const DataTypePt parser.parse(token_iterator, codec_ast, expected); - return CompressionCodecFactory::instance().get(codec_ast, data_type, false); + return CompressionCodecFactory::instance().get(codec_ast, data_type); } template From c09891b4f8b6c78eebbd1ed9acd08e9a921b5197 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Wed, 26 Aug 2020 02:12:51 +0300 Subject: [PATCH 0089/1911] DOCSUP-203: Update by PR#11558. --- docs/ru/operations/utilities/clickhouse-copier.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/utilities/clickhouse-copier.md b/docs/ru/operations/utilities/clickhouse-copier.md index b05db93b28b..b43f5ccaf7a 100644 --- a/docs/ru/operations/utilities/clickhouse-copier.md +++ b/docs/ru/operations/utilities/clickhouse-copier.md @@ -24,7 +24,7 @@ Утилиту следует запускать вручную следующим образом: ``` bash -$ clickhouse-copier copier --daemon --config zookeeper.xml --task-path /task/path --base-dir /path/to/dir +$ clickhouse-copier --daemon --config zookeeper.xml --task-path /task/path --base-dir /path/to/dir ``` Параметры запуска: From 3f53553522a34e1dd2312d8c7e85d9ae687f9df5 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Wed, 26 Aug 2020 02:37:32 +0300 Subject: [PATCH 0090/1911] DOCSUP-2031: Update by PR#11242. Added temporary_files_codec and join_on_disk_max_files_to_merge settings. --- docs/ru/operations/settings/settings.md | 25 +++++++++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index e8d3f1057df..ab64fb757f1 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -401,12 +401,33 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( Устанавливает тип поведения [JOIN](../../sql-reference/statements/select/join.md). При объединении таблиц могут появиться пустые ячейки. ClickHouse заполняет их по-разному в зависимости от настроек. -Возможные значения +Возможные значения: - 0 — пустые ячейки заполняются значением по умолчанию соответствующего типа поля. - 1 — `JOIN` ведёт себя как в стандартном SQL. Тип соответствующего поля преобразуется в [Nullable](../../sql-reference/data-types/nullable.md#data_type-nullable), а пустые ячейки заполняются значениями [NULL](../../sql-reference/syntax.md). -Значение по умолчанию: 0. +## join_on_disk_max_files_to_merge {#join_on_disk_max_files_to_merge} + +Устанавливет количество файлов, разрешенных для параллельной сортировки, при выполнении операций MergeJoin на диске. + +Чем больше значение параметра, тем больше оперативной памяти используется и тем меньше используется диск (I/O). + +Возможные значения: + +- Положительное целое число, больше 2. + +Значение по умолчанию: 64. + +## temporary_files_codec {#temporary_files_codec} + +Устанавливает метод сжатия для временных файлов на диске, используемых при сортировки и объединения. + +Возможные значения: + +- LZ4 — применять сжатие, используя алгоритм [LZ4](https://ru.wikipedia.org/wiki/LZ4) +- NONE — не применять сжатие. + +Значение по умолчанию: LZ4. ## max\_block\_size {#setting-max_block_size} From b0dd1d92b4be7f05da8637da88bbc3f35d07ee6b Mon Sep 17 00:00:00 2001 From: vladimir golovchenko Date: Wed, 12 Aug 2020 06:50:54 -0700 Subject: [PATCH 0091/1911] Added date_trunc function. --- src/Common/IntervalKind.cpp | 46 ++++++ src/Common/IntervalKind.h | 5 + src/Functions/date_trunc.cpp | 170 ++++++++++++++++++++ src/Functions/registerFunctionsDateTime.cpp | 2 + src/Functions/ya.make | 1 + 5 files changed, 224 insertions(+) create mode 100644 src/Functions/date_trunc.cpp diff --git a/src/Common/IntervalKind.cpp b/src/Common/IntervalKind.cpp index bcdae371bd4..b4acbe23ddd 100644 --- a/src/Common/IntervalKind.cpp +++ b/src/Common/IntervalKind.cpp @@ -176,4 +176,50 @@ const char * IntervalKind::toNameOfFunctionExtractTimePart() const } __builtin_unreachable(); } + + +bool IntervalKind::tryParseString(const std::string & kind, IntervalKind::Kind & result) +{ + if ("second" == kind) + { + result = IntervalKind::Second; + return true; + } + if ("minute" == kind) + { + result = IntervalKind::Minute; + return true; + } + if ("hour" == kind) + { + result = IntervalKind::Hour; + return true; + } + if ("day" == kind) + { + result = IntervalKind::Day; + return true; + } + if ("week" == kind) + { + result = IntervalKind::Week; + return true; + } + if ("month" == kind) + { + result = IntervalKind::Month; + return true; + } + if ("quarter" == kind) + { + result = IntervalKind::Quarter; + return true; + } + if ("year" == kind) + { + result = IntervalKind::Year; + return true; + } + return false; +} } diff --git a/src/Common/IntervalKind.h b/src/Common/IntervalKind.h index d8a569b8de4..91c3eb14043 100644 --- a/src/Common/IntervalKind.h +++ b/src/Common/IntervalKind.h @@ -52,5 +52,10 @@ struct IntervalKind /// For example, `IntervalKind{IntervalKind::Day}.getExtractTimePartFunctionName()` /// returns "toDayOfMonth". const char * toNameOfFunctionExtractTimePart() const; + + /// Converts the string representation of an interval kind to its IntervalKind equivalent. + /// Returns false if the conversion unsucceeded. + /// For example, `IntervalKind::tryParseString('second', result)` returns `result` equals `IntervalKind::Kind::Second`. + static bool tryParseString(const std::string & kind, IntervalKind::Kind & result); }; } diff --git a/src/Functions/date_trunc.cpp b/src/Functions/date_trunc.cpp new file mode 100644 index 00000000000..c2075de8f94 --- /dev/null +++ b/src/Functions/date_trunc.cpp @@ -0,0 +1,170 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +class FunctionDateTrunc : public IFunction +{ +public: + static constexpr auto name = "date_trunc"; + + explicit FunctionDateTrunc(const Context & context_) : context(context_) {} + + static FunctionPtr create(const Context & context) { return std::make_shared(context); } + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + /// The first argument is a constant string with the name of datepart. + + auto result_type_is_date = false; + String datepart_param; + auto check_first_argument = [&] { + const ColumnConst * datepart_column = checkAndGetColumnConst(arguments[0].column.get()); + if (!datepart_column) + throw Exception("First argument for function " + getName() + " must be constant string: name of datepart", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + datepart_param = datepart_column->getValue(); + if (datepart_param.empty()) + throw Exception("First argument (name of datepart) for function " + getName() + " cannot be empty", + ErrorCodes::BAD_ARGUMENTS); + + if (!IntervalKind::tryParseString(datepart_param, datepart_kind)) + throw Exception(datepart_param + " doesn't look like datepart name in " + getName(), + ErrorCodes::BAD_ARGUMENTS); + + result_type_is_date = (datepart_kind == IntervalKind::Year) + || (datepart_kind == IntervalKind::Quarter) || (datepart_kind == IntervalKind::Month) + || (datepart_kind == IntervalKind::Week); + }; + + bool second_argument_is_date = false; + auto check_second_argument = [&] { + if (!isDateOrDateTime(arguments[1].type)) + throw Exception( + "Illegal type " + arguments[1].type->getName() + " of 2nd argument of function " + getName() + + ". Should be a date or a date with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + second_argument_is_date = isDate(arguments[1].type); + + if (second_argument_is_date && ((datepart_kind == IntervalKind::Hour) + || (datepart_kind == IntervalKind::Minute) || (datepart_kind == IntervalKind::Second))) + throw Exception("Illegal type Date of argument for function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + }; + + auto check_timezone_argument = [&] { + if (!WhichDataType(arguments[2].type).isString()) + throw Exception( + "Illegal type " + arguments[2].type->getName() + " of argument of function " + getName() + + ". This argument is optional and must be a constant string with timezone name", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (second_argument_is_date && result_type_is_date) + throw Exception( + "The timezone argument of function " + getName() + " with datepart '" + datepart_param + + "' is allowed only when the 2nd argument has the type DateTime", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + }; + + if (arguments.size() == 2) + { + check_first_argument(); + check_second_argument(); + } + else if (arguments.size() == 3) + { + check_first_argument(); + check_second_argument(); + check_timezone_argument(); + } + else + { + throw Exception( + "Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + + ", should be 2 or 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + if (result_type_is_date) + return std::make_shared(); + else + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1)); + } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 2}; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override + { + Block temp_block = block; + + const UInt16 interval_value = 1; + const ColumnPtr interval_column = ColumnConst::create(ColumnInt64::create(1, interval_value), input_rows_count); + + const size_t interval_pos = temp_block.columns(); + temp_block.insert({interval_column, std::make_shared(datepart_kind), ""}); + + if (arguments.size() == 2) + { + auto to_start_of_interval = FunctionFactory::instance().get("toStartOfInterval", context)->build( + {temp_block.getByPosition(arguments[1]), temp_block.getByPosition(interval_pos)}); + + to_start_of_interval->execute(temp_block, {arguments[1], interval_pos}, result, input_rows_count); + } + else + { + auto to_start_of_interval = FunctionFactory::instance().get("toStartOfInterval", context)->build( + {temp_block.getByPosition(arguments[1]), temp_block.getByPosition(interval_pos), + temp_block.getByPosition(arguments[2])}); + + to_start_of_interval->execute(temp_block, {arguments[1], interval_pos, arguments[2]}, result, input_rows_count); + } + + block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column); + } + + bool hasInformationAboutMonotonicity() const override + { + return true; + } + + Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override + { + return { true, true, true }; + } + +private: + const Context & context; + mutable IntervalKind::Kind datepart_kind = IntervalKind::Kind::Second; +}; + + +void registerFunctionDateTrunc(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); + + /// Compatibility alias. + factory.registerAlias("dateTrunc", FunctionDateTrunc::name); +} + +} diff --git a/src/Functions/registerFunctionsDateTime.cpp b/src/Functions/registerFunctionsDateTime.cpp index 07c45241535..e95ab230087 100644 --- a/src/Functions/registerFunctionsDateTime.cpp +++ b/src/Functions/registerFunctionsDateTime.cpp @@ -65,6 +65,7 @@ void registerFunctionSubtractYears(FunctionFactory &); void registerFunctionDateDiff(FunctionFactory &); void registerFunctionToTimeZone(FunctionFactory &); void registerFunctionFormatDateTime(FunctionFactory &); +void registerFunctionDateTrunc(FunctionFactory &); void registerFunctionsDateTime(FunctionFactory & factory) { @@ -130,6 +131,7 @@ void registerFunctionsDateTime(FunctionFactory & factory) registerFunctionDateDiff(factory); registerFunctionToTimeZone(factory); registerFunctionFormatDateTime(factory); + registerFunctionDateTrunc(factory); } } diff --git a/src/Functions/ya.make b/src/Functions/ya.make index b6ab0d10f69..71c4fcfaf31 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -135,6 +135,7 @@ SRCS( currentDatabase.cpp currentUser.cpp dateDiff.cpp + date_trunc.cpp defaultValueOfArgumentType.cpp demange.cpp divide.cpp From f3cfd390035c24ae2d707f815454eaaa6114de31 Mon Sep 17 00:00:00 2001 From: vladimir golovchenko Date: Mon, 17 Aug 2020 02:45:26 -0700 Subject: [PATCH 0092/1911] Fixed .gitignore to exclude tests-generated data. --- programs/server/.gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/server/.gitignore b/programs/server/.gitignore index ba8a531c6d7..b774776e4be 100644 --- a/programs/server/.gitignore +++ b/programs/server/.gitignore @@ -4,6 +4,7 @@ /format_schemas /metadata_dropped /preprocessed_configs +/shadow /tmp /user_files From fb06f59ceb3dd61e3801e2f122e33e6efe41a1c6 Mon Sep 17 00:00:00 2001 From: vladimir golovchenko Date: Mon, 17 Aug 2020 02:47:16 -0700 Subject: [PATCH 0093/1911] Added tests for date_trunc function. --- tests/performance/date_time_long.xml | 1 + tests/performance/date_time_short.xml | 1 + .../0_stateless/00189_time_zones.reference | 55 ++++++++++++++++ .../queries/0_stateless/00189_time_zones.sql | 62 +++++++++++++++++++ .../00921_datetime64_compatibility.python | 4 ++ .../00921_datetime64_compatibility.reference | 21 +++++++ 6 files changed, 144 insertions(+) diff --git a/tests/performance/date_time_long.xml b/tests/performance/date_time_long.xml index b358945b35b..1229631a434 100644 --- a/tests/performance/date_time_long.xml +++ b/tests/performance/date_time_long.xml @@ -124,4 +124,5 @@ SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDate('2017-01-01') + number % 1000 + rand() % 10 AS t, {date_transform}(t)) SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, {binary_function}(t, 1)) SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, toStartOfInterval(t, INTERVAL 1 month)) + SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, date_trunc('month', t)) diff --git a/tests/performance/date_time_short.xml b/tests/performance/date_time_short.xml index a9fd0908694..05627ea482d 100644 --- a/tests/performance/date_time_short.xml +++ b/tests/performance/date_time_short.xml @@ -37,4 +37,5 @@ SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDate('2017-01-01') + number % 1000 + rand() % 10 AS t, {date_transform}(t)) SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, {binary_function}(t, 1)) SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, toStartOfInterval(t, INTERVAL 1 month)) + SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, date_trunc('month', t)) diff --git a/tests/queries/0_stateless/00189_time_zones.reference b/tests/queries/0_stateless/00189_time_zones.reference index 8e0abb19516..664c30056de 100644 --- a/tests/queries/0_stateless/00189_time_zones.reference +++ b/tests/queries/0_stateless/00189_time_zones.reference @@ -226,6 +226,7 @@ toDate 2014-09-30 2014-10-01 2014-09-30 +2014-09-30 toString 2015-07-15 13:30:00 2015-07-15 12:30:00 @@ -243,3 +244,57 @@ toUnixTimestamp 1426415400 1426415400 1426415400 +date_trunc +2019-01-01 +2020-01-01 +2020-01-01 +2019-10-01 +2020-01-01 +2020-01-01 +2019-12-01 +2020-01-01 +2020-01-01 +2019-12-30 +2019-12-30 +2019-12-30 +2019-12-31 00:00:00 +2020-01-01 00:00:00 +2020-01-02 00:00:00 +2019-12-31 20:00:00 +2020-01-01 12:00:00 +2020-01-02 05:00:00 +2019-12-31 20:11:00 +2020-01-01 12:11:00 +2020-01-02 05:11:00 +2019-12-31 20:11:22 +2020-01-01 12:11:22 +2020-01-02 05:11:22 +2019-01-01 +2020-01-01 +2020-01-01 +2019-10-01 +2020-01-01 +2020-01-01 +2019-12-01 +2020-01-01 +2020-01-01 +2019-12-30 +2019-12-30 +2019-12-30 +2019-12-31 00:00:00 +2020-01-01 00:00:00 +2020-01-02 00:00:00 +2019-12-31 20:00:00 +2020-01-01 12:00:00 +2020-01-02 05:00:00 +2019-12-31 20:11:00 +2020-01-01 12:11:00 +2020-01-02 05:11:00 +2019-12-31 20:11:22 +2020-01-01 12:11:22 +2020-01-02 05:11:22 +2020-01-01 +2020-01-01 +2020-01-01 +2019-12-30 +2020-01-01 00:00:00 diff --git a/tests/queries/0_stateless/00189_time_zones.sql b/tests/queries/0_stateless/00189_time_zones.sql index 45d8dc8e7fc..a0ef5b59517 100644 --- a/tests/queries/0_stateless/00189_time_zones.sql +++ b/tests/queries/0_stateless/00189_time_zones.sql @@ -321,6 +321,8 @@ SELECT toDate(1412106600, 'Europe/London'); SELECT toDate(1412106600, 'Asia/Tokyo'); SELECT toDate(1412106600, 'Pacific/Pitcairn'); +SELECT toDate(16343); + /* toString */ SELECT 'toString'; @@ -344,3 +346,63 @@ SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/Paris'), 'Europe SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/London'), 'Europe/London'); SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Asia/Tokyo'), 'Asia/Tokyo'); SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Pacific/Pitcairn'), 'Pacific/Pitcairn'); + +/* date_trunc */ + +SELECT 'date_trunc'; + +SELECT date_trunc('year', toDateTime('2020-01-01 04:11:22', 'Europe/London'), 'America/Vancouver'); +SELECT date_trunc('year', toDateTime('2020-01-01 12:11:22', 'Europe/London'), 'Europe/London'); +SELECT date_trunc('year', toDateTime('2020-01-01 20:11:22', 'Europe/London'), 'Asia/Tokyo'); +SELECT date_trunc('quarter', toDateTime('2020-01-01 04:11:22', 'Europe/London'), 'America/Vancouver'); +SELECT date_trunc('quarter', toDateTime('2020-01-01 12:11:22', 'Europe/London'), 'Europe/London'); +SELECT date_trunc('quarter', toDateTime('2020-01-01 20:11:22', 'Europe/London'), 'Asia/Tokyo'); +SELECT date_trunc('month', toDateTime('2020-01-01 04:11:22', 'Europe/London'), 'America/Vancouver'); +SELECT date_trunc('month', toDateTime('2020-01-01 12:11:22', 'Europe/London'), 'Europe/London'); +SELECT date_trunc('month', toDateTime('2020-01-01 20:11:22', 'Europe/London'), 'Asia/Tokyo'); +SELECT date_trunc('week', toDateTime('2020-01-01 04:11:22', 'Europe/London'), 'America/Vancouver'); +SELECT date_trunc('week', toDateTime('2020-01-01 12:11:22', 'Europe/London'), 'Europe/London'); +SELECT date_trunc('week', toDateTime('2020-01-01 20:11:22', 'Europe/London'), 'Asia/Tokyo'); +SELECT date_trunc('day', toDateTime('2020-01-01 04:11:22', 'Europe/London'), 'America/Vancouver'); +SELECT date_trunc('day', toDateTime('2020-01-01 12:11:22', 'Europe/London'), 'Europe/London'); +SELECT date_trunc('day', toDateTime('2020-01-01 20:11:22', 'Europe/London'), 'Asia/Tokyo'); +SELECT date_trunc('hour', toDateTime('2020-01-01 04:11:22', 'Europe/London'), 'America/Vancouver'); +SELECT date_trunc('hour', toDateTime('2020-01-01 12:11:22', 'Europe/London'), 'Europe/London'); +SELECT date_trunc('hour', toDateTime('2020-01-01 20:11:22', 'Europe/London'), 'Asia/Tokyo'); +SELECT date_trunc('minute', toDateTime('2020-01-01 04:11:22', 'Europe/London'), 'America/Vancouver'); +SELECT date_trunc('minute', toDateTime('2020-01-01 12:11:22', 'Europe/London'), 'Europe/London'); +SELECT date_trunc('minute', toDateTime('2020-01-01 20:11:22', 'Europe/London'), 'Asia/Tokyo'); +SELECT date_trunc('second', toDateTime('2020-01-01 04:11:22', 'Europe/London'), 'America/Vancouver'); +SELECT date_trunc('second', toDateTime('2020-01-01 12:11:22', 'Europe/London'), 'Europe/London'); +SELECT date_trunc('second', toDateTime('2020-01-01 20:11:22', 'Europe/London'), 'Asia/Tokyo'); + +SELECT date_trunc('year', toDateTime64('2020-01-01 04:11:22.123', 3, 'Europe/London'), 'America/Vancouver'); +SELECT date_trunc('year', toDateTime64('2020-01-01 12:11:22.123', 3, 'Europe/London'), 'Europe/London'); +SELECT date_trunc('year', toDateTime64('2020-01-01 20:11:22.123', 3, 'Europe/London'), 'Asia/Tokyo'); +SELECT date_trunc('quarter', toDateTime64('2020-01-01 04:11:22.123', 3, 'Europe/London'), 'America/Vancouver'); +SELECT date_trunc('quarter', toDateTime64('2020-01-01 12:11:22.123', 3, 'Europe/London'), 'Europe/London'); +SELECT date_trunc('quarter', toDateTime64('2020-01-01 20:11:22.123', 3, 'Europe/London'), 'Asia/Tokyo'); +SELECT date_trunc('month', toDateTime64('2020-01-01 04:11:22.123', 3, 'Europe/London'), 'America/Vancouver'); +SELECT date_trunc('month', toDateTime64('2020-01-01 12:11:22.123', 3, 'Europe/London'), 'Europe/London'); +SELECT date_trunc('month', toDateTime64('2020-01-01 20:11:22.123', 3, 'Europe/London'), 'Asia/Tokyo'); +SELECT date_trunc('week', toDateTime64('2020-01-01 04:11:22.123', 3, 'Europe/London'), 'America/Vancouver'); +SELECT date_trunc('week', toDateTime64('2020-01-01 12:11:22.123', 3, 'Europe/London'), 'Europe/London'); +SELECT date_trunc('week', toDateTime64('2020-01-01 20:11:22.123', 3, 'Europe/London'), 'Asia/Tokyo'); +SELECT date_trunc('day', toDateTime64('2020-01-01 04:11:22.123', 3, 'Europe/London'), 'America/Vancouver'); +SELECT date_trunc('day', toDateTime64('2020-01-01 12:11:22.123', 3, 'Europe/London'), 'Europe/London'); +SELECT date_trunc('day', toDateTime64('2020-01-01 20:11:22.123', 3, 'Europe/London'), 'Asia/Tokyo'); +SELECT date_trunc('hour', toDateTime64('2020-01-01 04:11:22.123', 3, 'Europe/London'), 'America/Vancouver'); +SELECT date_trunc('hour', toDateTime64('2020-01-01 12:11:22.123', 3, 'Europe/London'), 'Europe/London'); +SELECT date_trunc('hour', toDateTime64('2020-01-01 20:11:22.123', 3, 'Europe/London'), 'Asia/Tokyo'); +SELECT date_trunc('minute', toDateTime64('2020-01-01 04:11:22.123', 3, 'Europe/London'), 'America/Vancouver'); +SELECT date_trunc('minute', toDateTime64('2020-01-01 12:11:22.123', 3, 'Europe/London'), 'Europe/London'); +SELECT date_trunc('minute', toDateTime64('2020-01-01 20:11:22.123', 3, 'Europe/London'), 'Asia/Tokyo'); +SELECT date_trunc('second', toDateTime64('2020-01-01 04:11:22.123', 3, 'Europe/London'), 'America/Vancouver'); +SELECT date_trunc('second', toDateTime64('2020-01-01 12:11:22.123', 3, 'Europe/London'), 'Europe/London'); +SELECT date_trunc('second', toDateTime64('2020-01-01 20:11:22.123', 3, 'Europe/London'), 'Asia/Tokyo'); + +SELECT date_trunc('year', toDate('2020-01-01', 'Europe/London')); +SELECT date_trunc('quarter', toDate('2020-01-01', 'Europe/London')); +SELECT date_trunc('month', toDate('2020-01-01', 'Europe/London')); +SELECT date_trunc('week', toDate('2020-01-01', 'Europe/London')); +SELECT date_trunc('day', toDate('2020-01-01', 'Europe/London'), 'America/Vancouver'); diff --git a/tests/queries/0_stateless/00921_datetime64_compatibility.python b/tests/queries/0_stateless/00921_datetime64_compatibility.python index b67bec36947..54630755f05 100755 --- a/tests/queries/0_stateless/00921_datetime64_compatibility.python +++ b/tests/queries/0_stateless/00921_datetime64_compatibility.python @@ -35,6 +35,10 @@ toStartOfInterval(N, INTERVAL 1 year) toStartOfInterval(N, INTERVAL 1 month) toStartOfInterval(N, INTERVAL 1 day) toStartOfInterval(N, INTERVAL 15 minute) +date_trunc('year', N) +date_trunc('month', N) +date_trunc('day', N) +date_trunc('minute', N) toTime(N) toRelativeYearNum(N) toRelativeQuarterNum(N) diff --git a/tests/queries/0_stateless/00921_datetime64_compatibility.reference b/tests/queries/0_stateless/00921_datetime64_compatibility.reference index fa1ed7fc402..4042c756ba4 100644 --- a/tests/queries/0_stateless/00921_datetime64_compatibility.reference +++ b/tests/queries/0_stateless/00921_datetime64_compatibility.reference @@ -143,6 +143,27 @@ Code: 43: Illegal type Date of argument for function toStartOfInterval. "DateTime('Europe/Minsk')","2019-09-16 19:15:00" "DateTime('Europe/Minsk')","2019-09-16 19:15:00" ------------------------------------------ +SELECT date_trunc(\'year\', N) +"Date","2019-01-01" +"Date","2019-01-01" +"Date","2019-01-01" +------------------------------------------ +SELECT date_trunc(\'month\', N) +"Date","2019-09-01" +"Date","2019-09-01" +"Date","2019-09-01" +------------------------------------------ +SELECT date_trunc(\'day\', N) +"DateTime","2019-09-16 00:00:00" +"DateTime('Europe/Minsk')","2019-09-16 00:00:00" +"DateTime('Europe/Minsk')","2019-09-16 00:00:00" +------------------------------------------ +SELECT date_trunc(\'minute\', N) + +Code: 43: Illegal type Date of argument for function date_trunc. +"DateTime('Europe/Minsk')","2019-09-16 19:20:00" +"DateTime('Europe/Minsk')","2019-09-16 19:20:00" +------------------------------------------ SELECT toTime(N) Code: 43: Illegal type Date of argument for function toTime. From 962dd04ec086016c68c258fcbfa7bf6747e14d0f Mon Sep 17 00:00:00 2001 From: vladimir golovchenko Date: Wed, 19 Aug 2020 06:22:39 -0700 Subject: [PATCH 0094/1911] Added documentation for date_trunc function. --- .../functions/date-time-functions.md | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 0774f2176e8..c5986cd9ebc 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -339,6 +339,22 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d └────────────┴───────────┴───────────┴───────────┘ ``` +## date_trunc(datepart, time\_or\_data\[, time\_zone\]), dateTrunc(datepart, time\_or\_data\[, time\_zone\]) {#date_trunc} + +Truncates a date or date with time based on the specified datepart, such as +- `second` +- `minute` +- `hour` +- `day` +- `week` +- `month` +- `quarter` +- `year` + +```sql +SELECT date_trunc('hour', now()) +``` + ## now {#now} Accepts zero arguments and returns the current time at one of the moments of request execution. From c48d3b9d63f38e6a9f281b39060ab5e7bfbd5dfb Mon Sep 17 00:00:00 2001 From: 243f6a88 85a308d3 <33170174+243f6a8885a308d313198a2e037@users.noreply.github.com> Date: Wed, 26 Aug 2020 10:28:03 +0900 Subject: [PATCH 0095/1911] fixed Japanese translation for data-types/date.md --- docs/ja/sql-reference/data-types/date.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/ja/sql-reference/data-types/date.md b/docs/ja/sql-reference/data-types/date.md index ff6e028e885..528872d61c2 100644 --- a/docs/ja/sql-reference/data-types/date.md +++ b/docs/ja/sql-reference/data-types/date.md @@ -7,8 +7,7 @@ toc_title: "\u65E5\u4ED8" # 日付 {#date} -デートだ 1970-01-01(符号なし)以降の日数として二バイト単位で格納されます。 Unixエポックの開始直後から、コンパイル段階で定数によって定義される上限しきい値までの値を格納できます(現在は2106年までですが、完全にサポート -最小値は1970-01-01として出力されます。 +日付型です。 1970-01-01 からの日数が2バイトの符号なし整数として格納されます。 UNIX時間の開始直後から、変換段階で定数として定義される上限しきい値までの値を格納できます(現在は2106年までですが、一年分を完全にサポートしているのは2105年までです)。 日付値は、タイムゾーンなしで格納されます。 From cdcdb5a2c1f94cd629b2f1103340a6e08750c2fc Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 26 Aug 2020 10:47:00 +0300 Subject: [PATCH 0096/1911] Update date.md --- docs/ja/sql-reference/data-types/date.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/ja/sql-reference/data-types/date.md b/docs/ja/sql-reference/data-types/date.md index 528872d61c2..bcdc8f7224d 100644 --- a/docs/ja/sql-reference/data-types/date.md +++ b/docs/ja/sql-reference/data-types/date.md @@ -1,6 +1,4 @@ --- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd toc_priority: 47 toc_title: "\u65E5\u4ED8" --- From 21d78f85135954793da7d010cebc6d94faea8a08 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 26 Aug 2020 11:45:13 +0300 Subject: [PATCH 0097/1911] Better codec description representation --- src/Compression/CompressionCodecDelta.cpp | 7 ++++-- src/Compression/CompressionCodecDelta.h | 2 +- .../CompressionCodecDoubleDelta.cpp | 5 ++-- src/Compression/CompressionCodecDoubleDelta.h | 2 +- src/Compression/CompressionCodecGorilla.cpp | 5 ++-- src/Compression/CompressionCodecGorilla.h | 2 +- src/Compression/CompressionCodecLZ4.cpp | 12 ++++++---- src/Compression/CompressionCodecLZ4.h | 4 ++-- src/Compression/CompressionCodecMultiple.cpp | 21 +++++------------ src/Compression/CompressionCodecMultiple.h | 4 +--- src/Compression/CompressionCodecNone.cpp | 5 ++-- src/Compression/CompressionCodecNone.h | 2 +- src/Compression/CompressionCodecT64.cpp | 11 +++++++++ src/Compression/CompressionCodecT64.h | 6 ++--- src/Compression/CompressionCodecZSTD.cpp | 7 ++++-- src/Compression/CompressionCodecZSTD.h | 2 +- src/Compression/CompressionFactory.cpp | 23 +++++++++++-------- src/Compression/CompressionFactory.h | 19 +-------------- src/Compression/ICompressionCodec.h | 3 ++- src/Interpreters/InterpreterCreateQuery.cpp | 11 ++------- src/Interpreters/InterpreterDescribeQuery.cpp | 2 +- src/Storages/AlterCommands.cpp | 8 +++---- src/Storages/ColumnsDescription.cpp | 15 +++++------- src/Storages/ColumnsDescription.h | 2 +- src/Storages/System/StorageSystemColumns.cpp | 2 +- 25 files changed, 84 insertions(+), 98 deletions(-) diff --git a/src/Compression/CompressionCodecDelta.cpp b/src/Compression/CompressionCodecDelta.cpp index 7ecef677e5a..07945485dc3 100644 --- a/src/Compression/CompressionCodecDelta.cpp +++ b/src/Compression/CompressionCodecDelta.cpp @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include @@ -29,9 +31,10 @@ uint8_t CompressionCodecDelta::getMethodByte() const return static_cast(CompressionMethodByte::Delta); } -String CompressionCodecDelta::getCodecDesc() const +ASTPtr CompressionCodecDelta::getCodecDesc() const { - return fmt::format("Delta({})", size_t(delta_bytes_size)); + auto literal = std::make_shared(static_cast(delta_bytes_size)); + return makeASTFunction("Delta", literal); } namespace diff --git a/src/Compression/CompressionCodecDelta.h b/src/Compression/CompressionCodecDelta.h index ee3af359492..5c3979e063e 100644 --- a/src/Compression/CompressionCodecDelta.h +++ b/src/Compression/CompressionCodecDelta.h @@ -12,7 +12,7 @@ public: uint8_t getMethodByte() const override; - String getCodecDesc() const override; + ASTPtr getCodecDesc() const override; protected: UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index a9a69a50693..157e2df1a3f 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -333,9 +334,9 @@ uint8_t CompressionCodecDoubleDelta::getMethodByte() const return static_cast(CompressionMethodByte::DoubleDelta); } -String CompressionCodecDoubleDelta::getCodecDesc() const +ASTPtr CompressionCodecDoubleDelta::getCodecDesc() const { - return "DoubleDelta"; + return std::make_shared("DoubleDelta"); } UInt32 CompressionCodecDoubleDelta::getMaxCompressedDataSize(UInt32 uncompressed_size) const diff --git a/src/Compression/CompressionCodecDoubleDelta.h b/src/Compression/CompressionCodecDoubleDelta.h index dc3ec8bc243..a2690d24414 100644 --- a/src/Compression/CompressionCodecDoubleDelta.h +++ b/src/Compression/CompressionCodecDoubleDelta.h @@ -98,7 +98,7 @@ public: uint8_t getMethodByte() const override; - String getCodecDesc() const override; + ASTPtr getCodecDesc() const override; protected: UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; diff --git a/src/Compression/CompressionCodecGorilla.cpp b/src/Compression/CompressionCodecGorilla.cpp index b4f1cfc3c46..042835f4a32 100644 --- a/src/Compression/CompressionCodecGorilla.cpp +++ b/src/Compression/CompressionCodecGorilla.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -248,9 +249,9 @@ uint8_t CompressionCodecGorilla::getMethodByte() const return static_cast(CompressionMethodByte::Gorilla); } -String CompressionCodecGorilla::getCodecDesc() const +ASTPtr CompressionCodecGorilla::getCodecDesc() const { - return "Gorilla"; + return std::make_shared("Gorilla"); } UInt32 CompressionCodecGorilla::getMaxCompressedDataSize(UInt32 uncompressed_size) const diff --git a/src/Compression/CompressionCodecGorilla.h b/src/Compression/CompressionCodecGorilla.h index 48c2bdad1ee..523add0700f 100644 --- a/src/Compression/CompressionCodecGorilla.h +++ b/src/Compression/CompressionCodecGorilla.h @@ -95,7 +95,7 @@ public: uint8_t getMethodByte() const override; - String getCodecDesc() const override; + ASTPtr getCodecDesc() const override; protected: UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; diff --git a/src/Compression/CompressionCodecLZ4.cpp b/src/Compression/CompressionCodecLZ4.cpp index 32c3958e65e..cf3622cd702 100644 --- a/src/Compression/CompressionCodecLZ4.cpp +++ b/src/Compression/CompressionCodecLZ4.cpp @@ -7,6 +7,8 @@ #include #include #include +#include +#include #include #pragma GCC diagnostic ignored "-Wold-style-cast" @@ -28,9 +30,9 @@ uint8_t CompressionCodecLZ4::getMethodByte() const return static_cast(CompressionMethodByte::LZ4); } -String CompressionCodecLZ4::getCodecDesc() const +ASTPtr CompressionCodecLZ4::getCodecDesc() const { - return "LZ4"; + return std::make_shared("LZ4"); } UInt32 CompressionCodecLZ4::getMaxCompressedDataSize(UInt32 uncompressed_size) const @@ -56,10 +58,10 @@ void registerCodecLZ4(CompressionCodecFactory & factory) }); } - -String CompressionCodecLZ4HC::getCodecDesc() const +ASTPtr CompressionCodecLZ4HC::getCodecDesc() const { - return "LZ4HC(" + toString(level) + ")"; + auto literal = std::make_shared(static_cast(level)); + return makeASTFunction("LZ4HC", literal); } UInt32 CompressionCodecLZ4HC::doCompressData(const char * source, UInt32 source_size, char * dest) const diff --git a/src/Compression/CompressionCodecLZ4.h b/src/Compression/CompressionCodecLZ4.h index 0cfa694c416..2f19af08185 100644 --- a/src/Compression/CompressionCodecLZ4.h +++ b/src/Compression/CompressionCodecLZ4.h @@ -14,7 +14,7 @@ class CompressionCodecLZ4 : public ICompressionCodec public: uint8_t getMethodByte() const override; - String getCodecDesc() const override; + ASTPtr getCodecDesc() const override; UInt32 getAdditionalSizeAtTheEndOfBuffer() const override { return LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER; } @@ -39,7 +39,7 @@ public: CompressionCodecLZ4HC(int level_); - String getCodecDesc() const override; + ASTPtr getCodecDesc() const override; protected: UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; diff --git a/src/Compression/CompressionCodecMultiple.cpp b/src/Compression/CompressionCodecMultiple.cpp index 4b9a354d9a4..d762563b758 100644 --- a/src/Compression/CompressionCodecMultiple.cpp +++ b/src/Compression/CompressionCodecMultiple.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -28,22 +29,12 @@ uint8_t CompressionCodecMultiple::getMethodByte() const return static_cast(CompressionMethodByte::Multiple); } -String CompressionCodecMultiple::getCodecDesc() const +ASTPtr CompressionCodecMultiple::getCodecDesc() const { - return getCodecDescImpl(); -} - -String CompressionCodecMultiple::getCodecDescImpl() const -{ - WriteBufferFromOwnString out; - for (size_t idx = 0; idx < codecs.size(); ++idx) - { - if (idx != 0) - out << ", "; - - out << codecs[idx]->getCodecDesc(); - } - return out.str(); + auto result = std::make_shared(); + for (const auto & codec : codecs) + result->children.push_back(codec->getCodecDesc()); + return result; } UInt32 CompressionCodecMultiple::getMaxCompressedDataSize(UInt32 uncompressed_size) const diff --git a/src/Compression/CompressionCodecMultiple.h b/src/Compression/CompressionCodecMultiple.h index 8059b57bacb..dded559ca0b 100644 --- a/src/Compression/CompressionCodecMultiple.h +++ b/src/Compression/CompressionCodecMultiple.h @@ -13,7 +13,7 @@ public: uint8_t getMethodByte() const override; - String getCodecDesc() const override; + ASTPtr getCodecDesc() const override; UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; @@ -27,8 +27,6 @@ protected: private: Codecs codecs; - - String getCodecDescImpl() const; }; } diff --git a/src/Compression/CompressionCodecNone.cpp b/src/Compression/CompressionCodecNone.cpp index 3bee1c77276..50c19b2b547 100644 --- a/src/Compression/CompressionCodecNone.cpp +++ b/src/Compression/CompressionCodecNone.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB @@ -11,9 +12,9 @@ uint8_t CompressionCodecNone::getMethodByte() const return static_cast(CompressionMethodByte::NONE); } -String CompressionCodecNone::getCodecDesc() const +ASTPtr CompressionCodecNone::getCodecDesc() const { - return "NONE"; + return std::make_shared("NONE"); } UInt32 CompressionCodecNone::doCompressData(const char * source, UInt32 source_size, char * dest) const diff --git a/src/Compression/CompressionCodecNone.h b/src/Compression/CompressionCodecNone.h index 88bd8790810..ed604063198 100644 --- a/src/Compression/CompressionCodecNone.h +++ b/src/Compression/CompressionCodecNone.h @@ -13,7 +13,7 @@ class CompressionCodecNone : public ICompressionCodec public: uint8_t getMethodByte() const override; - String getCodecDesc() const override; + ASTPtr getCodecDesc() const override; protected: UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; diff --git a/src/Compression/CompressionCodecT64.cpp b/src/Compression/CompressionCodecT64.cpp index ba0adc8ec65..16462e50ebd 100644 --- a/src/Compression/CompressionCodecT64.cpp +++ b/src/Compression/CompressionCodecT64.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include @@ -635,6 +637,15 @@ uint8_t CompressionCodecT64::getMethodByte() const return codecId(); } +ASTPtr CompressionCodecT64::getCodecDesc() const +{ + if (variant == Variant::Byte) + return std::make_shared("T64"); + + auto literal = std::make_shared("bit"); + return makeASTFunction("T64", literal); +} + void registerCodecT64(CompressionCodecFactory & factory) { auto reg_func = [&](const ASTPtr & arguments, DataTypePtr type) -> CompressionCodecPtr diff --git a/src/Compression/CompressionCodecT64.h b/src/Compression/CompressionCodecT64.h index f5e142378df..11efbea0955 100644 --- a/src/Compression/CompressionCodecT64.h +++ b/src/Compression/CompressionCodecT64.h @@ -32,10 +32,8 @@ public: {} uint8_t getMethodByte() const override; - String getCodecDesc() const override - { - return variant == Variant::Byte ? "T64" : "T64('bit')"; - } + + ASTPtr getCodecDesc() const override; protected: UInt32 doCompressData(const char * src, UInt32 src_size, char * dst) const override; diff --git a/src/Compression/CompressionCodecZSTD.cpp b/src/Compression/CompressionCodecZSTD.cpp index 7abf0303095..ab48580533e 100644 --- a/src/Compression/CompressionCodecZSTD.cpp +++ b/src/Compression/CompressionCodecZSTD.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -24,9 +25,11 @@ uint8_t CompressionCodecZSTD::getMethodByte() const return static_cast(CompressionMethodByte::ZSTD); } -String CompressionCodecZSTD::getCodecDesc() const + +ASTPtr CompressionCodecZSTD::getCodecDesc() const { - return "ZSTD(" + toString(level) + ")"; + auto literal = std::make_shared(static_cast(level)); + return makeASTFunction("ZSTD", literal); } UInt32 CompressionCodecZSTD::getMaxCompressedDataSize(UInt32 uncompressed_size) const diff --git a/src/Compression/CompressionCodecZSTD.h b/src/Compression/CompressionCodecZSTD.h index d0790f2f2d7..2ad893083c3 100644 --- a/src/Compression/CompressionCodecZSTD.h +++ b/src/Compression/CompressionCodecZSTD.h @@ -17,7 +17,7 @@ public: uint8_t getMethodByte() const override; - String getCodecDesc() const override; + ASTPtr getCodecDesc() const override; UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 8c817b6bb37..5336959725a 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -48,20 +48,20 @@ void CompressionCodecFactory::validateCodec(const String & family_name, std::opt if (level) { auto literal = std::make_shared(static_cast(*level)); - validateCodecAndGetDescription(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), {}, sanity_check); + validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), {}, sanity_check); } else { auto identifier = std::make_shared(Poco::toUpper(family_name)); - validateCodecAndGetDescription(makeASTFunction("CODEC", identifier), {}, sanity_check); + validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier), {}, sanity_check); } } -CompressionCodecDescription CompressionCodecFactory::validateCodecAndGetDescription(const ASTPtr & ast, DataTypePtr column_type, bool sanity_check) const +ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr & ast, DataTypePtr column_type, bool sanity_check) const { if (const auto * func = ast->as()) { - Strings codecs_descriptions; + ASTPtr codecs_descriptions = std::make_shared(); bool is_compression = false; bool has_none = false; @@ -95,12 +95,12 @@ CompressionCodecDescription CompressionCodecFactory::validateCodecAndGetDescript "{} codec cannot have any arguments, it's just an alias for codec specified in config.xml", DEFAULT_CODEC_NAME); result_codec = default_codec; - codecs_descriptions.emplace_back(codec_family_name); + codecs_descriptions->children.emplace_back(std::make_shared(DEFAULT_CODEC_NAME)); } else { result_codec = getImpl(codec_family_name, codec_arguments, column_type); - codecs_descriptions.emplace_back(result_codec->getCodecDesc()); + codecs_descriptions->children.emplace_back(result_codec->getCodecDesc()); } is_compression |= result_codec->isCompression(); @@ -110,11 +110,11 @@ CompressionCodecDescription CompressionCodecFactory::validateCodecAndGetDescript generic_compression_codec_pos = i; } - String codec_description = boost::algorithm::join(codecs_descriptions, ", "); + String codec_description = queryToString(codecs_descriptions); if (sanity_check) { - if (codecs_descriptions.size() > 1 && has_none) + if (codecs_descriptions->children.size() > 1 && has_none) throw Exception( "It does not make sense to have codec NONE along with other compression codecs: " + codec_description + ". (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", @@ -134,13 +134,16 @@ CompressionCodecDescription CompressionCodecFactory::validateCodecAndGetDescript /// It does not make sense to apply any transformations after generic compression algorithm /// So, generic compression can be only one and only at the end. - if (generic_compression_codec_pos && *generic_compression_codec_pos != codecs_descriptions.size() - 1) + if (generic_compression_codec_pos && *generic_compression_codec_pos != codecs_descriptions->children.size() - 1) throw Exception("The combination of compression codecs " + codec_description + " is meaningless," " because it does not make sense to apply any transformations after generic compression algorithm." " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS); } - return CompressionCodecDescription{ast, codec_description}; + ASTPtr result = std::make_shared(); + result->as()->name = "CODEC"; + result->as()->arguments = codecs_descriptions; + return result; } throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC); diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index de5672365a9..f30050f81ba 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -20,23 +20,6 @@ using CompressionCodecPtr = std::shared_ptr; using CodecNameWithLevel = std::pair>; -/// Description for codec. AST and description may differ, because some codecs -/// may add some default parameters to their description. -/// -/// NOTE: We store both AST and preprocessed description. It would be simplier to use only AST -/// everywhere, but historically we store preprocessed codecs description in metadata files, -/// so now we cannot replace it with AST specified by user. -struct CompressionCodecDescription -{ - /// AST specified by user, for example "CODEC(Delta, LZ4HC)". Used to get - /// ICompressionCodec object from factory. - ASTPtr ast; - - /// String description with substituted codecs parameters: "Delta(4), LZ4HC(5)". - /// Stored in text representation of ColumnsDescription. - String description; -}; - /** Creates a codec object by name of compression algorithm family and parameters. */ class CompressionCodecFactory final : private boost::noncopyable @@ -55,7 +38,7 @@ public: CompressionCodecPtr getDefaultCodec() const; /// Validate codecs AST specified by user and parses codecs description (substitute default parameters) - CompressionCodecDescription validateCodecAndGetDescription(const ASTPtr & ast, DataTypePtr column_type, bool sanity_check) const; + ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, DataTypePtr column_type, bool sanity_check) const; /// Validate codecs AST specified by user void validateCodec(const String & family_name, std::optional level, bool sanity_check) const; diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index 664b88d7ac6..99fe53dd890 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -29,7 +30,7 @@ public: virtual uint8_t getMethodByte() const = 0; /// Codec description, for example "ZSTD(2)" or "LZ4,LZ4HC(5)" - virtual String getCodecDesc() const = 0; + virtual ASTPtr getCodecDesc() const = 0; /// Compressed bytes from uncompressed source to dest. Dest should preallocate memory UInt32 compress(const char * source, UInt32 source_size, char * dest) const; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 8a33275ba15..7aecdfa914a 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -281,14 +281,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) } if (column.codec) - { - String codec_desc = column.codec->description; - codec_desc = "CODEC(" + codec_desc + ")"; - const char * codec_desc_pos = codec_desc.data(); - const char * codec_desc_end = codec_desc_pos + codec_desc.size(); - ParserIdentifierWithParameters codec_p; - column_declaration->codec = parseQuery(codec_p, codec_desc_pos, codec_desc_end, "column codec", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - } + column_declaration->codec = column.codec; if (column.ttl) column_declaration->ttl = column.ttl; @@ -422,7 +415,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( column.comment = col_decl.comment->as().value.get(); if (col_decl.codec) - column.codec = CompressionCodecFactory::instance().validateCodecAndGetDescription(col_decl.codec, column.type, sanity_check_compression_codecs); + column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(col_decl.codec, column.type, sanity_check_compression_codecs); if (col_decl.ttl) column.ttl = col_decl.ttl; diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 48da2e33773..9c11bf147d3 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -116,7 +116,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() res_columns[4]->insert(column.comment); if (column.codec) - res_columns[5]->insert(column.codec->description); + res_columns[5]->insert(queryToString(column.codec->as()->arguments)); else res_columns[5]->insertDefault(); diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 497ee4feb7d..9280f07adcc 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -271,7 +271,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con column.comment = *comment; if (codec) - column.codec = CompressionCodecFactory::instance().validateCodecAndGetDescription(codec, data_type, false); + column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type, false); column.ttl = ttl; @@ -291,7 +291,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con metadata.columns.modify(column_name, after_column, first, [&](ColumnDescription & column) { if (codec) - column.codec = CompressionCodecFactory::instance().validateCodecAndGetDescription(codec, data_type ? data_type : column.type, false); + column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type ? data_type : column.type, false); if (comment) column.comment = *comment; @@ -815,7 +815,7 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con ErrorCodes::BAD_ARGUMENTS}; if (command.codec) - CompressionCodecFactory::instance().validateCodecAndGetDescription(command.codec, command.data_type, !context.getSettingsRef().allow_suspicious_codecs); + CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context.getSettingsRef().allow_suspicious_codecs); all_columns.add(ColumnDescription(column_name, command.data_type)); } @@ -835,7 +835,7 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con ErrorCodes::NOT_IMPLEMENTED}; if (command.codec) - CompressionCodecFactory::instance().validateCodecAndGetDescription(command.codec, command.data_type, !context.getSettingsRef().allow_suspicious_codecs); + CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context.getSettingsRef().allow_suspicious_codecs); modified_columns.emplace(column_name); } diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 41f88c5c2ae..bc800a47cc1 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -49,15 +49,14 @@ ColumnDescription::ColumnDescription(String name_, DataTypePtr type_) bool ColumnDescription::operator==(const ColumnDescription & other) const { - auto codec_str = [](const auto & codec_desc) { return codec_desc ? codec_desc->description : String{}; }; - auto ttl_str = [](const ASTPtr & ttl_ast) { return ttl_ast ? queryToString(ttl_ast) : String{}; }; + auto ast_to_str = [](const ASTPtr & ast) { return ast ? queryToString(ast) : String{}; }; return name == other.name && type->equals(*other.type) && default_desc == other.default_desc && comment == other.comment - && codec_str(codec) == codec_str(other.codec) - && ttl_str(ttl) == ttl_str(other.ttl); + && ast_to_str(codec) == ast_to_str(other.codec) + && ast_to_str(ttl) == ast_to_str(other.ttl); } void ColumnDescription::writeText(WriteBuffer & buf) const @@ -84,9 +83,7 @@ void ColumnDescription::writeText(WriteBuffer & buf) const if (codec) { writeChar('\t', buf); - DB::writeText("CODEC(", buf); - DB::writeText(codec->description, buf); - DB::writeText(")", buf); + DB::writeText(queryToString(codec), buf); } if (ttl) @@ -120,7 +117,7 @@ void ColumnDescription::readText(ReadBuffer & buf) comment = col_ast->comment->as().value.get(); if (col_ast->codec) - codec = CompressionCodecFactory::instance().validateCodecAndGetDescription(col_ast->codec, type, false); + codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(col_ast->codec, type, false); if (col_ast->ttl) ttl = col_ast->ttl; @@ -414,7 +411,7 @@ CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_ if (it == columns.get<1>().end() || !it->codec) return default_codec; - return CompressionCodecFactory::instance().get(it->codec->ast, it->type, default_codec); + return CompressionCodecFactory::instance().get(it->codec, it->type, default_codec); } CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_name) const diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index c9b4f6e38bd..145cadf8fb1 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -31,7 +31,7 @@ struct ColumnDescription DataTypePtr type; ColumnDefault default_desc; String comment; - std::optional codec; + ASTPtr codec; ASTPtr ttl; ColumnDescription() = default; diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index d714897a684..8a7368d5ace 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -214,7 +214,7 @@ protected: if (columns_mask[src_index++]) { if (column.codec) - res_columns[res_index++]->insert("CODEC(" + column.codec->description + ")"); + res_columns[res_index++]->insert(queryToString(column.codec)); else res_columns[res_index++]->insertDefault(); } From 249c4b4a94bb9aff5fa4d6051445b96247a5cb9c Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 26 Aug 2020 11:59:02 +0300 Subject: [PATCH 0098/1911] Less strange code and one method --- src/Compression/CompressionFactory.cpp | 6 +++--- src/Compression/ICompressionCodec.cpp | 8 ++++++++ src/Compression/ICompressionCodec.h | 4 ++++ 3 files changed, 15 insertions(+), 3 deletions(-) diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 5336959725a..cebeee7c5a8 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -140,9 +140,9 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS); } - ASTPtr result = std::make_shared(); - result->as()->name = "CODEC"; - result->as()->arguments = codecs_descriptions; + std::shared_ptr result = std::make_shared(); + result->name = "CODEC"; + result->arguments = codecs_descriptions; return result; } diff --git a/src/Compression/ICompressionCodec.cpp b/src/Compression/ICompressionCodec.cpp index 9a33def868b..64188165b5a 100644 --- a/src/Compression/ICompressionCodec.cpp +++ b/src/Compression/ICompressionCodec.cpp @@ -2,6 +2,7 @@ #include +#include #include #include @@ -15,6 +16,13 @@ namespace ErrorCodes extern const int CORRUPTED_DATA; } +ASTPtr ICompressionCodec::getFullCodecDesc() const +{ + std::shared_ptr result = std::make_shared(); + result->name = "CODEC"; + result->arguments = getCodecDesc(); + return result; +} UInt32 ICompressionCodec::compress(const char * source, UInt32 source_size, char * dest) const { diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index 99fe53dd890..fa1f73ce4dd 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -32,6 +32,10 @@ public: /// Codec description, for example "ZSTD(2)" or "LZ4,LZ4HC(5)" virtual ASTPtr getCodecDesc() const = 0; + /// Codec description with "CODEC" prefix, for example "CODEC(ZSTD(2))" or + /// "CODEC(LZ4,LZ4HC(5))" + ASTPtr getFullCodecDesc() const; + /// Compressed bytes from uncompressed source to dest. Dest should preallocate memory UInt32 compress(const char * source, UInt32 source_size, char * dest) const; From 4326c9c971c750b872a517fad5c971fb0b5018e3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 26 Aug 2020 12:16:32 +0300 Subject: [PATCH 0099/1911] Fix delta --- src/Compression/CompressionCodecDelta.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Compression/CompressionCodecDelta.cpp b/src/Compression/CompressionCodecDelta.cpp index 07945485dc3..51bd19f646b 100644 --- a/src/Compression/CompressionCodecDelta.cpp +++ b/src/Compression/CompressionCodecDelta.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include From 0f3351d983775eeee067d5d9d2e538238ed343bf Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 26 Aug 2020 13:22:08 +0300 Subject: [PATCH 0100/1911] Fix testflows checks. --- .../rbac/tests/syntax/grant_privilege.py | 47 ++++++++++--------- 1 file changed, 24 insertions(+), 23 deletions(-) diff --git a/tests/testflows/rbac/tests/syntax/grant_privilege.py b/tests/testflows/rbac/tests/syntax/grant_privilege.py index cabb3a3780b..82c459f546d 100755 --- a/tests/testflows/rbac/tests/syntax/grant_privilege.py +++ b/tests/testflows/rbac/tests/syntax/grant_privilege.py @@ -20,30 +20,30 @@ def setup(node): node.query("DROP ROLE IF EXISTS role1") @TestOutline(Scenario) -@Examples("privilege on allow_introspection", [ - ("dictGet", ("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_DictGet("1.0"))), - ("INTROSPECTION", ("*.*",), True, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Introspection("1.0"))), - ("SELECT", ("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Select("1.0"))), - ("INSERT",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Insert("1.0"))), - ("ALTER",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Alter("1.0"))), - ("CREATE",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Create("1.0"))), - ("DROP",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Drop("1.0"))), - ("TRUNCATE",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Truncate("1.0"))), - ("OPTIMIZE",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Optimize("1.0"))), - ("SHOW",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Show("1.0"))), - ("KILL QUERY",("*.*",), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_KillQuery("1.0"))), - ("ACCESS MANAGEMENT",("*.*",), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_AccessManagement("1.0"))), - ("SYSTEM",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_System("1.0"))), - ("SOURCES",("*.*",), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Sources("1.0"))), - ("ALL",("*.*",), True, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_All("1.0"))), - ("ALL PRIVILEGES",("*.*",), True, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_All("1.0"))), #alias for all +@Examples("privilege on allow_column allow_introspection", [ + ("dictGet", ("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_DictGet("1.0"))), + ("INTROSPECTION", ("*.*",), False, True, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Introspection("1.0"))), + ("SELECT", ("db0.table0","db0.*","*.*","tb0","*"), True, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Select("1.0"))), + ("INSERT",("db0.table0","db0.*","*.*","tb0","*"), True, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Insert("1.0"))), + ("ALTER",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Alter("1.0"))), + ("CREATE",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Create("1.0"))), + ("DROP",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Drop("1.0"))), + ("TRUNCATE",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Truncate("1.0"))), + ("OPTIMIZE",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Optimize("1.0"))), + ("SHOW",("db0.table0","db0.*","*.*","tb0","*"), True, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Show("1.0"))), + ("KILL QUERY",("*.*",), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_KillQuery("1.0"))), + ("ACCESS MANAGEMENT",("*.*",), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_AccessManagement("1.0"))), + ("SYSTEM",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_System("1.0"))), + ("SOURCES",("*.*",), False, False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_Sources("1.0"))), + ("ALL",("*.*",), True, True, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_All("1.0"))), + ("ALL PRIVILEGES",("*.*",), True, True, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_All("1.0"))), #alias for all ],) -def grant_privileges(self, privilege, on, allow_introspection, node="clickhouse1"): - grant_privilege(privilege=privilege, on=on, allow_introspection=allow_introspection, node=node) +def grant_privileges(self, privilege, on, allow_column, allow_introspection, node="clickhouse1"): + grant_privilege(privilege=privilege, on=on, allow_column=allow_column, allow_introspection=allow_introspection, node=node) @TestOutline(Scenario) @Requirements(RQ_SRS_006_RBAC_Grant_Privilege_GrantOption("1.0")) -def grant_privilege(self, privilege, on, allow_introspection, node="clickhouse1"): +def grant_privilege(self, privilege, on, allow_column, allow_introspection, node="clickhouse1"): node = self.context.cluster.node(node) for on_ in on: @@ -58,9 +58,10 @@ def grant_privilege(self, privilege, on, allow_introspection, node="clickhouse1" with When("I grant privilege with grant option"): node.query(f"GRANT {privilege} ON {on_} TO user1 WITH GRANT OPTION", settings=settings) - #grant column specific for some column 'x' - with When("I grant privilege with columns"): - node.query(f"GRANT {privilege}(x) ON {on_} TO user0", settings=settings) + if allow_column and ('*' not in on_): + #grant column specific for some column 'x' + with When("I grant privilege with columns"): + node.query(f"GRANT {privilege}(x) ON {on_} TO user0", settings=settings) @TestFeature @Name("grant privilege") From 7ac4bd7d1efe26a7693e72752696092704483e4a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 21 Aug 2020 18:47:37 +0300 Subject: [PATCH 0101/1911] Add storages from after ones from and . --- src/Access/AccessControlManager.cpp | 44 ++++++++----------- .../configs/local_directories.xml | 2 + .../test_user_directories/configs/memory.xml | 3 ++ .../configs/mixed_style.xml | 8 ++++ .../configs/old_style.xml | 1 + .../configs/relative_path.xml | 3 ++ .../integration/test_user_directories/test.py | 8 ++++ 7 files changed, 43 insertions(+), 26 deletions(-) create mode 100644 tests/integration/test_user_directories/configs/mixed_style.xml diff --git a/src/Access/AccessControlManager.cpp b/src/Access/AccessControlManager.cpp index 6158be1b603..1fa26c85354 100644 --- a/src/Access/AccessControlManager.cpp +++ b/src/Access/AccessControlManager.cpp @@ -281,41 +281,33 @@ void AccessControlManager::addStoragesFromMainConfig( String config_dir = std::filesystem::path{config_path}.remove_filename().string(); String dbms_dir = config.getString("path", DBMS_DEFAULT_PATH); String include_from_path = config.getString("include_from", "/etc/metrika.xml"); + bool has_user_directories = config.has("user_directories"); - if (config.has("user_directories")) + /// If path to users' config isn't absolute, try guess its root (current) dir. + /// At first, try to find it in dir of main config, after will use current dir. + String users_config_path = config.getString("users_config", ""); + if (users_config_path.empty()) { - if (config.has("users_config")) - LOG_WARNING(getLogger(), " is specified, the path from won't be used: " + config.getString("users_config")); - if (config.has("access_control_path")) - LOG_WARNING(getLogger(), " is specified, the path from won't be used: " + config.getString("access_control_path")); - - addStoragesFromUserDirectoriesConfig( - config, - "user_directories", - config_dir, - dbms_dir, - include_from_path, - get_zookeeper_function); - } - else - { - /// If path to users' config isn't absolute, try guess its root (current) dir. - /// At first, try to find it in dir of main config, after will use current dir. - String users_config_path = config.getString("users_config", ""); - if (users_config_path.empty()) + if (!has_user_directories) users_config_path = config_path; - else if (std::filesystem::path{users_config_path}.is_relative() && std::filesystem::exists(config_dir + users_config_path)) - users_config_path = config_dir + users_config_path; + } + else if (std::filesystem::path{users_config_path}.is_relative() && std::filesystem::exists(config_dir + users_config_path)) + users_config_path = config_dir + users_config_path; + if (!users_config_path.empty()) + { if (users_config_path != config_path) checkForUsersNotInMainConfig(config, config_path, users_config_path, getLogger()); addUsersConfigStorage(users_config_path, include_from_path, dbms_dir, get_zookeeper_function); - - String disk_storage_dir = config.getString("access_control_path", ""); - if (!disk_storage_dir.empty()) - addDiskStorage(disk_storage_dir); } + + String disk_storage_dir = config.getString("access_control_path", ""); + if (!disk_storage_dir.empty()) + addDiskStorage(disk_storage_dir); + + if (has_user_directories) + addStoragesFromUserDirectoriesConfig(config, "user_directories", config_dir, dbms_dir, include_from_path, get_zookeeper_function); } diff --git a/tests/integration/test_user_directories/configs/local_directories.xml b/tests/integration/test_user_directories/configs/local_directories.xml index e2cbcd135df..7b9601da982 100644 --- a/tests/integration/test_user_directories/configs/local_directories.xml +++ b/tests/integration/test_user_directories/configs/local_directories.xml @@ -12,4 +12,6 @@ /var/lib/clickhouse/access3-ro/ + +
diff --git a/tests/integration/test_user_directories/configs/memory.xml b/tests/integration/test_user_directories/configs/memory.xml index 6e906d2b1d6..78da38ed0bc 100644 --- a/tests/integration/test_user_directories/configs/memory.xml +++ b/tests/integration/test_user_directories/configs/memory.xml @@ -5,4 +5,7 @@ + + + diff --git a/tests/integration/test_user_directories/configs/mixed_style.xml b/tests/integration/test_user_directories/configs/mixed_style.xml new file mode 100644 index 00000000000..d6ddecf6f5d --- /dev/null +++ b/tests/integration/test_user_directories/configs/mixed_style.xml @@ -0,0 +1,8 @@ + + + + + + /etc/clickhouse-server/users6.xml + /var/lib/clickhouse/access6/ + diff --git a/tests/integration/test_user_directories/configs/old_style.xml b/tests/integration/test_user_directories/configs/old_style.xml index a0ff36edaba..cc753006b22 100644 --- a/tests/integration/test_user_directories/configs/old_style.xml +++ b/tests/integration/test_user_directories/configs/old_style.xml @@ -1,5 +1,6 @@ /etc/clickhouse-server/users2.xml /var/lib/clickhouse/access2/ + diff --git a/tests/integration/test_user_directories/configs/relative_path.xml b/tests/integration/test_user_directories/configs/relative_path.xml index 8906478959e..c4ef3c5fd79 100644 --- a/tests/integration/test_user_directories/configs/relative_path.xml +++ b/tests/integration/test_user_directories/configs/relative_path.xml @@ -4,4 +4,7 @@ users4.xml + + + diff --git a/tests/integration/test_user_directories/test.py b/tests/integration/test_user_directories/test.py index 8b7f34cf999..218330cb1a5 100644 --- a/tests/integration/test_user_directories/test.py +++ b/tests/integration/test_user_directories/test.py @@ -16,6 +16,7 @@ def started_cluster(): node.exec_in_container("cp /etc/clickhouse-server/users.xml /etc/clickhouse-server/users3.xml") node.exec_in_container("cp /etc/clickhouse-server/users.xml /etc/clickhouse-server/users4.xml") node.exec_in_container("cp /etc/clickhouse-server/users.xml /etc/clickhouse-server/users5.xml") + node.exec_in_container("cp /etc/clickhouse-server/users.xml /etc/clickhouse-server/users6.xml") yield cluster @@ -49,3 +50,10 @@ def test_memory(): node.restart_clickhouse() assert node.query("SELECT * FROM system.user_directories") == TSV([["users.xml", "users.xml", "/etc/clickhouse-server/users5.xml", 1, 1], ["memory", "memory", "", 0, 2]]) + +def test_mixed_style(): + node.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/mixed_style.xml"), '/etc/clickhouse-server/config.d/z.xml') + node.restart_clickhouse() + assert node.query("SELECT * FROM system.user_directories") == TSV([["users.xml", "users.xml", "/etc/clickhouse-server/users6.xml", 1, 1], + ["local directory", "local directory", "/var/lib/clickhouse/access6/", 0, 2], + ["memory", "memory", "", 0, 3]]) From da16f234cc0cef5e93f7d47f3b0b166496c2b48f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 26 Aug 2020 14:06:32 +0300 Subject: [PATCH 0102/1911] Update CacheDictionary.cpp --- src/Dictionaries/CacheDictionary.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 87ccb2af9ce..b223e1aec0d 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -798,10 +798,9 @@ void CacheDictionary::waitForCurrentUpdateFinish(UpdateUnitPtr & update_unit_ptr { std::unique_lock update_lock(update_mutex); - size_t timeout_for_wait = 100000; bool result = is_update_finished.wait_for( update_lock, - std::chrono::milliseconds(timeout_for_wait), + std::chrono::milliseconds(query_wait_timeout_milliseconds), [&] { return update_unit_ptr->is_done || update_unit_ptr->current_exception; }); if (!result) @@ -817,7 +816,7 @@ void CacheDictionary::waitForCurrentUpdateFinish(UpdateUnitPtr & update_unit_ptr * */ update_unit_ptr->can_use_callback = false; throw DB::Exception(ErrorCodes::TIMEOUT_EXCEEDED, - "Dictionary {} source seems unavailable, because {} timeout exceeded.", + "Dictionary {} source seems unavailable, because {}ms timeout exceeded.", getDictionaryID().getNameForLogs(), toString(timeout_for_wait)); } From fe5db661cedb0e6f8fdb092780e7c74a6a5e3242 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Wed, 26 Aug 2020 15:26:12 +0300 Subject: [PATCH 0103/1911] fix decimal to float conversion and toDecimal256() --- .../AggregateFunctionStatisticsSimple.h | 55 +++++++++---------- src/Core/DecimalFunctions.h | 5 +- src/DataTypes/DataTypeDecimalBase.h | 10 ++-- src/DataTypes/DataTypesDecimal.h | 6 ++ src/Formats/ProtobufWriter.cpp | 2 +- src/Functions/FunctionMathUnary.h | 2 +- src/Functions/FunctionsConversion.h | 22 ++------ tests/performance/decimal_casts.xml | 27 +++++++++ 8 files changed, 74 insertions(+), 55 deletions(-) create mode 100644 tests/performance/decimal_casts.xml diff --git a/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h b/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h index cdcfd902a1e..e23c220dd3d 100644 --- a/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h +++ b/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h @@ -130,37 +130,25 @@ struct VarMoments }; template -struct VarMomentsDecimal +class VarMomentsDecimal { +public: using NativeType = typename T::NativeType; - UInt64 m0{}; - NativeType m[_level]{}; - - NativeType & getM(size_t i) - { - return m[i - 1]; - } - - const NativeType & getM(size_t i) const - { - return m[i - 1]; - } - void add(NativeType x) { ++m0; getM(1) += x; NativeType tmp; - if (common::mulOverflow(x, x, tmp) || common::addOverflow(getM(2), tmp, getM(2))) - throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); + bool overflow = common::mulOverflow(x, x, tmp) || common::addOverflow(getM(2), tmp, getM(2)); if constexpr (_level >= 3) - if (common::mulOverflow(tmp, x, tmp) || common::addOverflow(getM(3), tmp, getM(3))) - throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); + overflow = overflow || common::mulOverflow(tmp, x, tmp) || common::addOverflow(getM(3), tmp, getM(3)); if constexpr (_level >= 4) - if (common::mulOverflow(tmp, x, tmp) || common::addOverflow(getM(4), tmp, getM(4))) - throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); + overflow = overflow || common::mulOverflow(tmp, x, tmp) || common::addOverflow(getM(4), tmp, getM(4)); + + if (overflow) + throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); } void merge(const VarMomentsDecimal & rhs) @@ -168,14 +156,14 @@ struct VarMomentsDecimal m0 += rhs.m0; getM(1) += rhs.getM(1); - if (common::addOverflow(getM(2), rhs.getM(2), getM(2))) - throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); + bool overflow = common::addOverflow(getM(2), rhs.getM(2), getM(2)); if constexpr (_level >= 3) - if (common::addOverflow(getM(3), rhs.getM(3), getM(3))) - throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); + overflow = overflow || common::addOverflow(getM(3), rhs.getM(3), getM(3)); if constexpr (_level >= 4) - if (common::addOverflow(getM(4), rhs.getM(4), getM(4))) - throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); + overflow = overflow || common::addOverflow(getM(4), rhs.getM(4), getM(4)); + + if (overflow) + throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); } void write(WriteBuffer & buf) const { writePODBinary(*this, buf); } @@ -190,7 +178,7 @@ struct VarMomentsDecimal if (common::mulOverflow(getM(1), getM(1), tmp) || common::subOverflow(getM(2), NativeType(tmp / m0), tmp)) throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); - return std::max(Float64{}, convertFromDecimal, DataTypeNumber>(tmp / m0, scale)); + return std::max(Float64{}, DecimalUtils::convertTo(T(tmp / m0), scale)); } Float64 getSample(UInt32 scale) const @@ -204,7 +192,7 @@ struct VarMomentsDecimal if (common::mulOverflow(getM(1), getM(1), tmp) || common::subOverflow(getM(2), NativeType(tmp / m0), tmp)) throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); - return std::max(Float64{}, convertFromDecimal, DataTypeNumber>(tmp / (m0 - 1), scale)); + return std::max(Float64{}, DecimalUtils::convertTo(T(tmp / (m0 - 1)), scale)); } Float64 getMoment3(UInt32 scale) const @@ -218,7 +206,7 @@ struct VarMomentsDecimal common::mulOverflow(tmp, getM(1), tmp) || common::subOverflow(getM(3), NativeType(tmp / m0), tmp)) throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); - return convertFromDecimal, DataTypeNumber>(tmp / m0, scale); + return DecimalUtils::convertTo(T(tmp / m0), scale); } Float64 getMoment4(UInt32 scale) const @@ -234,8 +222,15 @@ struct VarMomentsDecimal common::mulOverflow(tmp, getM(1), tmp) || common::subOverflow(getM(4), NativeType(tmp / m0), tmp)) throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); - return convertFromDecimal, DataTypeNumber>(tmp / m0, scale); + return DecimalUtils::convertTo(T(tmp / m0), scale); } + +private: + UInt64 m0{}; + NativeType m[_level]{}; + + NativeType & getM(size_t i) { return m[i - 1]; } + const NativeType & getM(size_t i) const { return m[i - 1]; } }; /** diff --git a/src/Core/DecimalFunctions.h b/src/Core/DecimalFunctions.h index 899ab9fe5e0..ca1c44d01cd 100644 --- a/src/Core/DecimalFunctions.h +++ b/src/Core/DecimalFunctions.h @@ -183,7 +183,10 @@ To convertTo(const DecimalType & decimal, size_t scale) if constexpr (std::is_floating_point_v) { - return static_cast(decimal.value) / static_cast(scaleMultiplier(scale)); + if constexpr (is_big_int_v) + return static_cast(decimal.value) / static_cast(scaleMultiplier(scale)); + else + return static_cast(decimal.value) / scaleMultiplier(scale); } else if constexpr (is_integer_v && (sizeof(To) >= sizeof(NativeT))) { diff --git a/src/DataTypes/DataTypeDecimalBase.h b/src/DataTypes/DataTypeDecimalBase.h index 53836221e12..265d58d69e1 100644 --- a/src/DataTypes/DataTypeDecimalBase.h +++ b/src/DataTypes/DataTypeDecimalBase.h @@ -158,7 +158,7 @@ protected: template typename DecimalType> typename std::enable_if_t<(sizeof(T) >= sizeof(U)), DecimalType> -decimalResultType(const DecimalType & tx, const DecimalType & ty, bool is_multiply, bool is_divide) +inline decimalResultType(const DecimalType & tx, const DecimalType & ty, bool is_multiply, bool is_divide) { UInt32 scale = (tx.getScale() > ty.getScale() ? tx.getScale() : ty.getScale()); if (is_multiply) @@ -170,7 +170,7 @@ decimalResultType(const DecimalType & tx, const DecimalType & ty, bool is_ template typename DecimalType> typename std::enable_if_t<(sizeof(T) < sizeof(U)), const DecimalType> -decimalResultType(const DecimalType & tx, const DecimalType & ty, bool is_multiply, bool is_divide) +inline decimalResultType(const DecimalType & tx, const DecimalType & ty, bool is_multiply, bool is_divide) { UInt32 scale = (tx.getScale() > ty.getScale() ? tx.getScale() : ty.getScale()); if (is_multiply) @@ -181,19 +181,19 @@ decimalResultType(const DecimalType & tx, const DecimalType & ty, bool is_ } template typename DecimalType> -const DecimalType decimalResultType(const DecimalType & tx, const DataTypeNumber &, bool, bool) +inline const DecimalType decimalResultType(const DecimalType & tx, const DataTypeNumber &, bool, bool) { return DecimalType(DecimalUtils::maxPrecision(), tx.getScale()); } template typename DecimalType> -const DecimalType decimalResultType(const DataTypeNumber &, const DecimalType & ty, bool, bool) +inline const DecimalType decimalResultType(const DataTypeNumber &, const DecimalType & ty, bool, bool) { return DecimalType(DecimalUtils::maxPrecision(), ty.getScale()); } template