From e44e1ad0d4cb7b08e4b1de3cf863f060e4d493c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jun 2020 21:57:38 +0300 Subject: [PATCH 01/68] 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 02/68] 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 03/68] 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 88db4938f5ec53d343d76789db2d084cb84b5e1f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Aug 2020 02:22:00 +0300 Subject: [PATCH 04/68] Fix error; refinements --- .../ReplicatedMergeTreeBlockOutputStream.cpp | 65 +++++++++++-------- 1 file changed, 37 insertions(+), 28 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 4aa8b12bd96..2b2570e0187 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -218,6 +218,11 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( String temporary_part_name = part->name; + /// There is one case when we need to retry transaction in a loop. + /// But don't do it too many times - just as defensive measure. + size_t loop_counter = 0; + constexpr size_t max_iterations = 10; + while (true) { /// Obtain incremental block number and lock it. The lock holds our intention to add the block to the filesystem. @@ -229,6 +234,10 @@ 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); + /// Prepare transaction to ZooKeeper + /// It will simultaneously add information about the part to all the necessary places in ZooKeeper and remove block_number_lock. + Coordination::Requests ops; + Int64 block_number = 0; String existing_part_name; if (block_number_lock) @@ -242,6 +251,25 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( part->info.level = 0; part->name = part->getNewName(part->info); + + /// Will add log entry about new part. + + 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; + log_entry.new_part_type = part->getType(); + + 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); } else { @@ -269,43 +297,21 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( part->name = existing_part_name; part->info = MergeTreePartInfo::fromPartName(existing_part_name, storage.format_version); + /// Used only for exception messages. block_number = part->info.min_block; /// Don't do subsequent duplicate check. block_id_path.clear(); } - 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; - log_entry.new_part_type = part->getType(); - - /// 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 we 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; @@ -405,6 +411,9 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( part->state = MergeTreeDataPartState::Temporary; part->renameTo(temporary_part_name, false); + ++loop_counter; + if (loop_counter == max_iterations) + throw Exception("Too many transaction retires - it may indicate an error", ErrorCodes::DUPLICATE_DATA_PART); continue; } else if (multi_code == Coordination::Error::ZNODEEXISTS && failed_op_path == quorum_info.status_path) From cdba5e727c71680344a1c20a1378635bc6194695 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Aug 2020 02:30:07 +0300 Subject: [PATCH 05/68] Fix mistake --- .../ReplicatedMergeTreeBlockOutputStream.cpp | 83 +++++++++---------- 1 file changed, 40 insertions(+), 43 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 2b2570e0187..f856f936982 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -270,6 +270,46 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( /// Deletes the information that the block number is used for writing. block_number_lock->getUnlockOps(ops); + + /** If we 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)); + } } else { @@ -299,54 +339,11 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( /// Used only for exception messages. block_number = part->info.min_block; - - /// Don't do subsequent duplicate check. - block_id_path.clear(); } /// Information about the part. storage.getCommitPartOps(ops, part, block_id_path); - /** If we 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 From 6082697c4d7775d9777f7b33d4583ce73fdb25f6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Aug 2020 02:39:12 +0300 Subject: [PATCH 06/68] Support for quorum --- .../MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index f856f936982..e62cbf95b59 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -223,6 +223,8 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( size_t loop_counter = 0; constexpr size_t max_iterations = 10; + bool is_already_existing_part = false; + while (true) { /// Obtain incremental block number and lock it. The lock holds our intention to add the block to the filesystem. @@ -242,6 +244,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( String existing_part_name; if (block_number_lock) { + is_already_existing_part = false; block_number = block_number_lock->getNumber(); /// Set part attributes according to part_number. Prepare an entry for log. @@ -313,6 +316,8 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( } else { + is_already_existing_part = true; + /// 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); @@ -446,6 +451,11 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( if (quorum) { + if (is_already_existing_part) + { + storage.updateQuorum(part->name); + } + /// We are waiting for quorum to be satisfied. LOG_TRACE(log, "Waiting for quorum"); From c4e8aaac166e0d6350ccf59fc5d750191544a5e9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Aug 2020 03:07:51 +0300 Subject: [PATCH 07/68] Fixups --- .../ReplicatedMergeTreeBlockOutputStream.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index e62cbf95b59..0f0674f66ed 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -216,7 +216,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( metadata_snapshot->check(part->getColumns()); assertSessionIsNotExpired(zookeeper); - String temporary_part_name = part->name; + String temporary_part_relative_path = part->relative_path; /// There is one case when we need to retry transaction in a loop. /// But don't do it too many times - just as defensive measure. @@ -344,6 +344,9 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( /// Used only for exception messages. block_number = part->info.min_block; + + /// Do not check for duplicate on commit to ZK. + block_id_path.clear(); } /// Information about the part. @@ -362,7 +365,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( } if (!renamed) { - if (!existing_part_name.empty()) + if (is_already_existing_part) { LOG_INFO(log, "Part {} is duplicate and it is already written by concurrent request; ignoring it.", block_id, existing_part_name); return; @@ -404,14 +407,14 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( { /// 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); + block_id, part->name, temporary_part_relative_path); transaction.rollback(); part->is_duplicate = true; part->is_temp = true; part->state = MergeTreeDataPartState::Temporary; - part->renameTo(temporary_part_name, false); + part->renameTo(temporary_part_relative_path, false); ++loop_counter; if (loop_counter == max_iterations) From ed1d120de0a34f5c1664056d2098f72da050169f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Aug 2020 03:28:37 +0300 Subject: [PATCH 08/68] Added another test; fixup --- .../ReplicatedMergeTreeBlockOutputStream.cpp | 20 +++++++--- .../01459_manual_write_to_replicas.reference | 2 + .../01459_manual_write_to_replicas.sh | 38 +++++++++++++++++++ 3 files changed, 54 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/01459_manual_write_to_replicas.reference create mode 100755 tests/queries/0_stateless/01459_manual_write_to_replicas.sh diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 0f0674f66ed..196ec6586ae 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 PART_IS_TEMPORARILY_LOCKED; extern const int LOGICAL_ERROR; } @@ -98,7 +99,8 @@ void ReplicatedMergeTreeBlockOutputStream::checkQuorumPrecondition(zkutil::ZooKe auto quorum_status = quorum_status_future.get(); if (quorum_status.error != Coordination::Error::ZNONODE) - throw Exception("Quorum for previous write has not been satisfied yet. Status: " + quorum_status.data, ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE); + throw Exception("Quorum for previous write has not been satisfied yet. Status: " + quorum_status.data, + ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE); /// Both checks are implicitly made also later (otherwise there would be a race condition). @@ -305,7 +307,8 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( 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. + /// 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( @@ -360,18 +363,22 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( } catch (const Exception & e) { - if (e.code() != ErrorCodes::DUPLICATE_DATA_PART) + if (e.code() != ErrorCodes::DUPLICATE_DATA_PART + && e.code() != ErrorCodes::PART_IS_TEMPORARILY_LOCKED) throw; } if (!renamed) { if (is_already_existing_part) { - LOG_INFO(log, "Part {} is duplicate and it is already written by concurrent request; ignoring it.", block_id, existing_part_name); + LOG_INFO(log, "Part {} is duplicate and it is already written by concurrent request or fetched; 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); + 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; @@ -485,7 +492,8 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( throw Exception("Timeout while waiting for quorum", ErrorCodes::TIMEOUT_EXCEEDED); } - /// And what if it is possible that the current replica at this time has ceased to be active and the quorum is marked as failed and deleted? + /// And what if it is possible that the current replica at this time has ceased to be active + /// and the quorum is marked as failed and deleted? String value; if (!zookeeper->tryGet(storage.replica_path + "/is_active", value, nullptr) || value != quorum_info.is_active_node_value) diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas.reference b/tests/queries/0_stateless/01459_manual_write_to_replicas.reference new file mode 100644 index 00000000000..b8d8ae420e0 --- /dev/null +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas.reference @@ -0,0 +1,2 @@ +100 0 99 4950 +100 0 99 4950 diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh new file mode 100755 index 00000000000..d8c955c40c0 --- /dev/null +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash + +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -n -q " + DROP TABLE IF EXISTS r1; + DROP TABLE IF EXISTS r2; + + CREATE TABLE r1 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/r', 'r1') ORDER BY x; + CREATE TABLE r2 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/r', 'r2') ORDER BY x; +" + +function thread { + for x in {0..99}; do + $CLICKHOUSE_CLIENT --query "INSERT INTO r$1 SELECT $x" + done +} + +thread 1 & +thread 2 & + +wait + +$CLICKHOUSE_CLIENT -n -q " + SYSTEM SYNC REPLICA r1; + SYSTEM SYNC REPLICA r2; +" + +$CLICKHOUSE_CLIENT -q "SELECT count(), min(x), max(x), sum(x) FROM r1"; +$CLICKHOUSE_CLIENT -q "SELECT count(), min(x), max(x), sum(x) FROM r2"; + +$CLICKHOUSE_CLIENT -n -q " + DROP TABLE IF EXISTS r1; + DROP TABLE IF EXISTS r2; +" From 5763737d97a95b51d764d0396bd49ac1a29532c9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Aug 2020 03:53:22 +0300 Subject: [PATCH 09/68] Fixups --- src/Storages/MergeTree/MergeTreeData.cpp | 3 ++- src/Storages/StorageReplicatedMergeTree.cpp | 18 ++++++++++++++++-- src/Storages/StorageReplicatedMergeTree.h | 7 ++++++- 3 files changed, 24 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 36c02c4db1f..9d4b1a7a041 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3262,7 +3262,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk( } if (!does_storage_policy_allow_same_disk) throw Exception( - "Could not clone and load part " + quoteString(src_part->getFullPath()) + " because disk does not belong to storage policy", ErrorCodes::BAD_ARGUMENTS); + "Could not clone and load part " + quoteString(src_part->getFullPath()) + " because disk does not belong to storage policy", + ErrorCodes::BAD_ARGUMENTS); String dst_part_name = src_part->getNewName(dst_part_info); String tmp_dst_part_name = tmp_part_prefix + dst_part_name; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 93dee1905e2..cf1889f443f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -113,6 +113,7 @@ namespace ErrorCodes extern const int ALL_REPLICAS_LOST; extern const int REPLICA_STATUS_CHANGED; extern const int CANNOT_ASSIGN_ALTER; + extern const int DIRECTORY_ALREADY_EXISTS; } namespace ActionLocks @@ -3256,6 +3257,15 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora part->renameTo("detached/" + part_name, true); } } + catch (const Exception & e) + { + /// The same part is being written right now (but probably it's not committed yet). + /// We will check the need for fetch later. + if (e.code() == ErrorCodes::DIRECTORY_ALREADY_EXISTS) + return false; + + throw; + } catch (...) { if (!to_detached) @@ -4689,9 +4699,11 @@ void StorageReplicatedMergeTree::fetchPartition( missing_parts.clear(); for (const String & part : parts_to_fetch) { + bool fetched = false; + try { - fetchPart(part, metadata_snapshot, best_replica_path, true, 0); + fetched = fetchPart(part, metadata_snapshot, best_replica_path, true, 0); } catch (const DB::Exception & e) { @@ -4700,8 +4712,10 @@ void StorageReplicatedMergeTree::fetchPartition( throw; LOG_INFO(log, e.displayText()); - missing_parts.push_back(part); } + + if (!fetched) + missing_parts.push_back(part); } ++try_no; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index ad992a10f08..1d50687e18a 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -478,7 +478,12 @@ private: * If quorum != 0, then the node for tracking the quorum is updated. * Returns false if part is already fetching right now. */ - bool fetchPart(const String & part_name, const StorageMetadataPtr & metadata_snapshot, const String & replica_path, bool to_detached, size_t quorum); + bool fetchPart( + const String & part_name, + const StorageMetadataPtr & metadata_snapshot, + const String & replica_path, + bool to_detached, + size_t quorum); /// Required only to avoid races between executeLogEntry and fetchPartition std::unordered_set currently_fetching_parts; From 538b7730ddbb1d28b9c0819e507f7f045b3c7f67 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Aug 2020 04:02:04 +0300 Subject: [PATCH 10/68] Use 10 replicas in test --- .../01459_manual_write_to_replicas.reference | 8 +++++ .../01459_manual_write_to_replicas.sh | 35 +++++++++---------- 2 files changed, 24 insertions(+), 19 deletions(-) diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas.reference b/tests/queries/0_stateless/01459_manual_write_to_replicas.reference index b8d8ae420e0..52dea650ebc 100644 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas.reference +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas.reference @@ -1,2 +1,10 @@ 100 0 99 4950 100 0 99 4950 +100 0 99 4950 +100 0 99 4950 +100 0 99 4950 +100 0 99 4950 +100 0 99 4950 +100 0 99 4950 +100 0 99 4950 +100 0 99 4950 diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh index d8c955c40c0..17da6e73a14 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh @@ -5,13 +5,14 @@ set -e CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n -q " - DROP TABLE IF EXISTS r1; - DROP TABLE IF EXISTS r2; +NUM_REPLICAS=10 - CREATE TABLE r1 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/r', 'r1') ORDER BY x; - CREATE TABLE r2 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/r', 'r2') ORDER BY x; -" +for i in $(seq 1 $NUM_REPLICAS); do + $CLICKHOUSE_CLIENT -n -q " + DROP TABLE IF EXISTS r$i; + CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/r', 'r$i') ORDER BY x; + " +done function thread { for x in {0..99}; do @@ -19,20 +20,16 @@ function thread { done } -thread 1 & -thread 2 & +for i in $(seq 1 $NUM_REPLICAS); do + thread $i & +done wait -$CLICKHOUSE_CLIENT -n -q " - SYSTEM SYNC REPLICA r1; - SYSTEM SYNC REPLICA r2; -" - -$CLICKHOUSE_CLIENT -q "SELECT count(), min(x), max(x), sum(x) FROM r1"; -$CLICKHOUSE_CLIENT -q "SELECT count(), min(x), max(x), sum(x) FROM r2"; - -$CLICKHOUSE_CLIENT -n -q " - DROP TABLE IF EXISTS r1; - DROP TABLE IF EXISTS r2; +for i in $(seq 1 $NUM_REPLICAS); do + $CLICKHOUSE_CLIENT -n -q " + SYSTEM SYNC REPLICA r$i; + SELECT count(), min(x), max(x), sum(x) FROM r$i; + DROP TABLE IF EXISTS r$i; " +done From 733446a5be2b99c9d29b07d20b4db23e9169d60d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Aug 2020 04:05:28 +0300 Subject: [PATCH 11/68] Added a test with quorum --- ..._manual_write_to_replicas_quorum.reference | 10 +++++ .../01459_manual_write_to_replicas_quorum.sh | 37 +++++++++++++++++++ 2 files changed, 47 insertions(+) create mode 100644 tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.reference create mode 100755 tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.reference b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.reference new file mode 100644 index 00000000000..52dea650ebc --- /dev/null +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.reference @@ -0,0 +1,10 @@ +100 0 99 4950 +100 0 99 4950 +100 0 99 4950 +100 0 99 4950 +100 0 99 4950 +100 0 99 4950 +100 0 99 4950 +100 0 99 4950 +100 0 99 4950 +100 0 99 4950 diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh new file mode 100755 index 00000000000..e6709b76316 --- /dev/null +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh @@ -0,0 +1,37 @@ +#!/usr/bin/env bash + +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +NUM_REPLICAS=10 + +for i in $(seq 1 $NUM_REPLICAS); do + $CLICKHOUSE_CLIENT -n -q " + DROP TABLE IF EXISTS r$i; + CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/r', 'r$i') ORDER BY x; + " +done + +function thread { + for x in {0..99}; do + while true; do + $CLICKHOUSE_CLIENT --insert_quorum 5 --query "INSERT INTO r$1 SELECT $x" 2>&1 | grep -qF 'Quorum for previous write has not been satisfied yet' || break + done + done +} + +for i in $(seq 1 $NUM_REPLICAS); do + thread $i & +done + +wait + +for i in $(seq 1 $NUM_REPLICAS); do + $CLICKHOUSE_CLIENT -n -q " + SYSTEM SYNC REPLICA r$i; + SELECT count(), min(x), max(x), sum(x) FROM r$i; + DROP TABLE IF EXISTS r$i; +" +done From dd6e23bbbd2acf99d2de0709997cbf4bfee9f01f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Aug 2020 04:08:09 +0300 Subject: [PATCH 12/68] Slightly better test --- tests/queries/0_stateless/01459_manual_write_to_replicas.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh index 17da6e73a14..42f5dc1bbf3 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh @@ -16,7 +16,7 @@ done function thread { for x in {0..99}; do - $CLICKHOUSE_CLIENT --query "INSERT INTO r$1 SELECT $x" + $CLICKHOUSE_CLIENT --query "INSERT INTO r$1 SELECT $x % $NUM_REPLICAS = $1 ? $x - 1 : $x" # Replace some records as duplicates so they will be written by other replicas done } From 0db5b4a72ceb27a39b79c4b975f6c119e9057e29 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 2 Sep 2020 02:43:23 +0300 Subject: [PATCH 13/68] Remove concurrent benchmark from the perf test It's not stable enough and has only secondary utility. --- docker/test/performance-comparison/compare.sh | 33 +-------- docker/test/performance-comparison/report.py | 67 ------------------- 2 files changed, 3 insertions(+), 97 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 4384f5b7827..d8e3dc93442 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -197,33 +197,9 @@ function run_tests wait } -# Run some queries concurrently and report the resulting TPS. This additional -# (relatively) short test helps detect concurrency-related effects, because the -# main performance comparison testing is done query-by-query. -function run_benchmark -{ - rm -rf benchmark ||: - mkdir benchmark ||: - - # The list is built by run_tests. - while IFS= read -r file - do - name=$(basename "$file" ".xml") - - "$script_dir/perf.py" --print-queries "$file" > "benchmark/$name-queries.txt" - "$script_dir/perf.py" --print-settings "$file" > "benchmark/$name-settings.txt" - - readarray -t settings < "benchmark/$name-settings.txt" - command=(clickhouse-benchmark --concurrency 6 --cumulative --iterations 1000 --randomize 1 --delay 0 --continue_on_errors "${settings[@]}") - - "${command[@]}" --port 9001 --json "benchmark/$name-left.json" < "benchmark/$name-queries.txt" - "${command[@]}" --port 9002 --json "benchmark/$name-right.json" < "benchmark/$name-queries.txt" - done < benchmarks-to-run.txt -} - function get_profiles_watchdog { - sleep 6000 + sleep 600 echo "The trace collection did not finish in time." >> profile-errors.log @@ -570,8 +546,8 @@ create table test_time_changes engine File(TSV, 'report/test-time-changes.tsv') select test, count(*) queries, sum(left) as left, sum(right) as right, (right - left) / right average_time_change - from queries - group by test + from queries + group by test order by abs(average_time_change) desc ) ; @@ -980,9 +956,6 @@ case "$stage" in # Ignore the errors to collect the log and build at least some report, anyway time run_tests ||: ;& -"run_benchmark") - time run_benchmark 2> >(tee -a run-errors.tsv 1>&2) ||: - ;& "get_profiles") # Check for huge pages. cat /sys/kernel/mm/transparent_hugepage/enabled > thp-enabled.txt ||: diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index d7fc2a9707b..4529718df51 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -497,73 +497,6 @@ if args.report == 'main': add_test_times() - def add_benchmark_results(): - if not os.path.isfile('benchmark/website-left.json'): - return - - json_reports = [json.load(open(f'benchmark/website-{x}.json')) for x in ['left', 'right']] - stats = [next(iter(x.values()))["statistics"] for x in json_reports] - qps = [x["QPS"] for x in stats] - queries = [x["num_queries"] for x in stats] - errors = [x["num_errors"] for x in stats] - relative_diff = (qps[1] - qps[0]) / max(0.01, qps[0]); - times_diff = max(qps) / max(0.01, min(qps)) - - all_rows = [] - header = ['Benchmark', 'Metric', 'Old', 'New', 'Relative difference', 'Times difference']; - - attrs = ['' for x in header] - row = ['website', 'queries', f'{queries[0]:d}', f'{queries[1]:d}', '--', '--'] - attrs[0] = 'rowspan=2' - all_rows.append([row, attrs]) - - attrs = ['' for x in header] - row = [None, 'queries/s', f'{qps[0]:.3f}', f'{qps[1]:.3f}', f'{relative_diff:.3f}', f'x{times_diff:.3f}'] - if abs(relative_diff) > 0.1: - # More queries per second is better. - if relative_diff > 0.: - attrs[4] = f'style="background: {color_good}"' - else: - attrs[4] = f'style="background: {color_bad}"' - else: - attrs[4] = '' - all_rows.append([row, attrs]); - - if max(errors): - all_rows[0][1][0] = "rowspan=3" - row = [''] * (len(header)) - attrs = ['' for x in header] - - attrs[0] = None - row[1] = 'errors' - row[2] = f'{errors[0]:d}' - row[3] = f'{errors[1]:d}' - row[4] = '--' - row[5] = '--' - if errors[0]: - attrs[2] += f' style="background: {color_bad}" ' - if errors[1]: - attrs[3] += f' style="background: {color_bad}" ' - - all_rows.append([row, attrs]) - - text = tableStart('Concurrent benchmarks') - text += tableHeader(header) - for row, attrs in all_rows: - text += tableRow(row, attrs) - text += tableEnd() - - global tables - tables.append(text) - - try: - add_benchmark_results() - except: - report_errors.append( - traceback.format_exception_only( - *sys.exc_info()[:2])[-1]) - pass - addSimpleTable('Metric changes', ['Metric', 'Old median value', 'New median value', 'Relative difference', 'Times difference'], From e622e108f7f3a7c2cdd246086c1ee3c6cd119423 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 2 Sep 2020 06:29:16 +0300 Subject: [PATCH 14/68] readme --- docker/test/performance-comparison/README.md | 31 +++++++++----------- 1 file changed, 14 insertions(+), 17 deletions(-) diff --git a/docker/test/performance-comparison/README.md b/docker/test/performance-comparison/README.md index d877f435c24..d91cd9421ea 100644 --- a/docker/test/performance-comparison/README.md +++ b/docker/test/performance-comparison/README.md @@ -16,7 +16,7 @@ We also consider the test to be unstable, if the observed difference is less tha performance differences above 5% more often than in 5% runs, so the test is likely to have false positives. -### How to read the report +### How to Read the Report The check status summarizes the report in a short text message like `1 faster, 10 unstable`: * `1 faster` -- how many queries became faster, @@ -27,27 +27,27 @@ The check status summarizes the report in a short text message like `1 faster, 1 The report page itself constists of a several tables. Some of them always signify errors, e.g. "Run errors" -- the very presence of this table indicates that there were errors during the test, that are not normal and must be fixed. Some tables are mostly informational, e.g. "Test times" -- they reflect normal test results. But if a cell in such table is marked in red, this also means an error, e.g., a test is taking too long to run. -#### Tested commits +#### Tested Commits Informational, no action required. Log messages for the commits that are tested. Note that for the right commit, we show nominal tested commit `pull/*/head` and real tested commit `pull/*/merge`, which is generated by GitHub by merging latest master to the `pull/*/head` and which we actually build and test in CI. -#### Run errors +#### Run Errors Action required for every item -- these are errors that must be fixed. The errors that ocurred when running some test queries. For more information about the error, download test output archive and see `test-name-err.log`. To reproduce, see 'How to run' below. -#### Slow on client +#### Slow on Client Action required for every item -- these are errors that must be fixed. This table shows queries that take significantly longer to process on the client than on the server. A possible reason might be sending too much data to the client, e.g., a forgotten `format Null`. -#### Short queries not marked as short +#### Short Queries not Marked as Short Action required for every item -- these are errors that must be fixed. This table shows queries that are "short" but not explicitly marked as such. "Short" queries are too fast to meaningfully compare performance, because the changes are drowned by the noise. We consider all queries that run faster than 0.02 s to be "short", and only check the performance if they became slower than this threshold. Probably this mode is not what you want, so you have to increase the query run time to be between 1 and 0.1 s, so that the performance can be compared. You do want this "short" mode for queries that complete "immediately", such as some varieties of `select count(*)`. You have to mark them as "short" explicitly by writing `...`. The value of "short" attribute is evaluated as a python expression, and substitutions are performed, so you can write something like `select count(*) from table where {column1} > {column2}`, to mark only a particular combination of variables as short. -#### Partial queries +#### Partial Queries Action required for the cells marked in red. Shows the queries we are unable to run on an old server -- probably because they contain a new function. You should see this table when you add a new function and a performance test for it. Check that the run time and variance are acceptable (run time between 0.1 and 1 seconds, variance below 10%). If not, they will be highlighted in red. -#### Changes in performance +#### Changes in Performance Action required for the cells marked in red, and some cheering is appropriate for the cells marked in green. These are the queries for which we observe a statistically significant change in performance. Note that there will always be some false positives -- we try to filter by p < 0.001, and have 2000 queries, so two false positives per run are expected. In practice we have more -- e.g. code layout changed because of some unknowable jitter in compiler internals, so the change we observe is real, but it is a 'false positive' in the sense that it is not directly caused by your changes. If, based on your knowledge of ClickHouse internals, you can decide that the observed test changes are not relevant to the changes made in the tested PR, you can ignore them. You can find flame graphs for queries with performance changes in the test output archive, in files named as 'my_test_0_Cpu_SELECT 1 FROM....FORMAT Null.left.svg'. First goes the test name, then the query number in the test, then the trace type (same as in `system.trace_log`), and then the server version (left is old and right is new). -#### Unstable queries +#### Unstable Queries Action required for the cells marked in red. These are queries for which we did not observe a statistically significant change in performance, but for which the variance in query performance is very high. This means that we are likely to observe big changes in performance even in the absence of real changes, e.g. when comparing the server to itself. Such queries are going to have bad sensitivity as performance tests -- if a query has, say, 50% expected variability, this means we are going to see changes in performance up to 50%, even when there were no real changes in the code. And because of this, we won't be able to detect changes less than 50% with such a query, which is pretty bad. The reasons for the high variability must be investigated and fixed; ideally, the variability should be brought under 5-10%. The most frequent reason for instability is that the query is just too short -- e.g. below 0.1 seconds. Bringing query time to 0.2 seconds or above usually helps. @@ -57,24 +57,21 @@ Other reasons may include: Investigating the instablility is the hardest problem in performance testing, and we still have not been able to understand the reasons behind the instability of some queries. There are some data that can help you in the performance test output archive. Look for files named 'my_unstable_test_0_SELECT 1...FORMAT Null.{left,right}.metrics.rep'. They contain metrics from `system.query_log.ProfileEvents` and functions from stack traces from `system.trace_log`, that vary significantly between query runs. The second column is array of \[min, med, max] values for the metric. Say, if you see `PerfCacheMisses` there, it may mean that the code being tested has not-so-cache-local memory access pattern that is sensitive to memory layout. -#### Skipped tests +#### Skipped Tests Informational, no action required. Shows the tests that were skipped, and the reason for it. Normally it is because the data set required for the test was not loaded, or the test is marked as 'long' -- both cases mean that the test is too big to be ran per-commit. -#### Test performance changes +#### Test Performance Changes Informational, no action required. This table summarizes the changes in performance of queries in each test -- how many queries have changed, how many are unstable, and what is the magnitude of the changes. -#### Test times +#### Test Times Action required for the cells marked in red. This table shows the run times for all the tests. You may have to fix two kinds of errors in this table: 1) Average query run time is too long -- probalby means that the preparatory steps such as creating the table and filling them with data are taking too long. Try to make them faster. 2) Longest query run time is too long -- some particular queries are taking too long, try to make them faster. The ideal query run time is between 0.1 and 1 s. -#### Concurrent benchmarks -No action required. This table shows the results of a concurrent behcmark where queries from `website` are ran in parallel using `clickhouse-benchmark`, and requests per second values are compared for old and new servers. It shows variability up to 20% for no apparent reason, so it's probably safe to disregard it. We have it for special cases like investigating concurrency effects in memory allocators, where it may be important. +#### Metric Changes +No action required. These are changes in median values of metrics from `system.asynchronous_metrics_log`. These metrics are prone to unexplained variation and you can safely ignore this table unless it's interesting to you for some particular reason (e.g. you want to compare memory usage). There are also graphs of these metrics in the performance test output archive, in the `metrics` folder. -#### Metric changes -No action required. These are changes in median values of metrics from `system.asynchronous_metrics_log`. Again, they are prone to unexplained variation and you can safely ignore this table unless it's interesting to you for some particular reason (e.g. you want to compare memory usage). There are also graphs of these metrics in the performance test output archive, in the `metrics` folder. - -### How to run +### How to Run Run the entire docker container, specifying PR number (0 for master) and SHA of the commit to test. The reference revision is determined as a nearest ancestor testing release tag. It is possible to specify the reference revision and From 5fcb9bd1635bc3cb1742ca8518471c6b9dabe1b5 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 2 Sep 2020 06:29:36 +0300 Subject: [PATCH 15/68] short queries --- docker/test/performance-comparison/perf.py | 84 ++++++++++++++-------- 1 file changed, 54 insertions(+), 30 deletions(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index a659326b068..fe8aff6b4cb 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -1,16 +1,19 @@ #!/usr/bin/python3 -import os -import sys -import itertools -import clickhouse_driver -import xml.etree.ElementTree as et import argparse +import clickhouse_driver +import itertools +import functools +import math +import os import pprint import re +import statistics import string +import sys import time import traceback +import xml.etree.ElementTree as et def tsv_escape(s): return s.replace('\\', '\\\\').replace('\t', '\\t').replace('\n', '\\n').replace('\r','') @@ -62,18 +65,13 @@ def substitute_parameters(query_templates, other_templates = []): # Build a list of test queries, substituting parameters to query templates, # and reporting the queries marked as short. test_queries = [] +is_short = [] for e in root.findall('query'): - new_queries = [] - if 'short' in e.attrib: - new_queries, [is_short] = substitute_parameters([e.text], [[e.attrib['short']]]) - for i, s in enumerate(is_short): - # Don't print this if we only need to print the queries. - if eval(s) and not args.print_queries: - print(f'short\t{i + len(test_queries)}') - else: - new_queries = substitute_parameters([e.text]) - + new_queries, [new_is_short] = substitute_parameters([e.text], [[e.attrib.get('short', '0')]]) test_queries += new_queries + is_short += [eval(s) for s in new_is_short] + +assert(len(test_queries) == len(is_short)) # If we're only asked to print the queries, do that and exit @@ -82,6 +80,11 @@ if args.print_queries: print(q) exit(0) +# Print short queries +for i, s in enumerate(is_short): + if s: + print(f'short\t{i}') + # If we're only asked to print the settings, do that and exit. These are settings # for clickhouse-benchmark, so we print them as command line arguments, e.g. # '--max_memory_usage=10000000'. @@ -116,7 +119,7 @@ if 'max_ignored_relative_change' in root.attrib: # Open connections servers = [{'host': host, 'port': port} for (host, port) in zip(args.host, args.port)] -connections = [clickhouse_driver.Client(**server) for server in servers] +all_connections = [clickhouse_driver.Client(**server) for server in servers] for s in servers: print('server\t{}\t{}'.format(s['host'], s['port'])) @@ -126,7 +129,7 @@ for s in servers: # connection loses the changes in settings. drop_query_templates = [q.text for q in root.findall('drop_query')] drop_queries = substitute_parameters(drop_query_templates) -for conn_index, c in enumerate(connections): +for conn_index, c in enumerate(all_connections): for q in drop_queries: try: c.execute(q) @@ -142,7 +145,7 @@ for conn_index, c in enumerate(connections): # configurable). So the end result is uncertain, but hopefully we'll be able to # run at least some queries. settings = root.findall('settings/*') -for conn_index, c in enumerate(connections): +for conn_index, c in enumerate(all_connections): for s in settings: try: q = f"set {s.tag} = '{s.text}'" @@ -154,7 +157,7 @@ for conn_index, c in enumerate(connections): # Check tables that should exist. If they don't exist, just skip this test. tables = [e.text for e in root.findall('preconditions/table_exists')] for t in tables: - for c in connections: + for c in all_connections: try: res = c.execute("select 1 from {} limit 1".format(t)) except: @@ -176,7 +179,7 @@ for q in create_queries: file = sys.stderr) sys.exit(1) -for conn_index, c in enumerate(connections): +for conn_index, c in enumerate(all_connections): for q in create_queries: c.execute(q) print(f'create\t{conn_index}\t{c.last_query.elapsed}\t{tsv_escape(q)}') @@ -184,7 +187,7 @@ for conn_index, c in enumerate(connections): # Run fill queries fill_query_templates = [q.text for q in root.findall('fill_query')] fill_queries = substitute_parameters(fill_query_templates) -for conn_index, c in enumerate(connections): +for conn_index, c in enumerate(all_connections): for q in fill_queries: c.execute(q) print(f'fill\t{conn_index}\t{c.last_query.elapsed}\t{tsv_escape(q)}') @@ -208,8 +211,8 @@ for query_index, q in enumerate(test_queries): # new one. We want to run them on the new server only, so that the PR author # can ensure that the test works properly. Remember the errors we had on # each server. - query_error_on_connection = [None] * len(connections); - for conn_index, c in enumerate(connections): + query_error_on_connection = [None] * len(all_connections); + for conn_index, c in enumerate(all_connections): try: prewarm_id = f'{query_prefix}.prewarm0' res = c.execute(q, query_id = prewarm_id) @@ -236,21 +239,22 @@ for query_index, q in enumerate(test_queries): if len(no_errors) == 0: continue - elif len(no_errors) < len(connections): + elif len(no_errors) < len(all_connections): print(f'partial\t{query_index}\t{no_errors}') + this_query_connections = [all_connections[index] for index in no_errors] + # Now, perform measured runs. # Track the time spent by the client to process this query, so that we can # notice the queries that take long to process on the client side, e.g. by # sending excessive data. start_seconds = time.perf_counter() server_seconds = 0 - for run in range(0, args.runs): + run = 0 + while True: run_id = f'{query_prefix}.run{run}' - for conn_index, c in enumerate(connections): - if query_error_on_connection[conn_index]: - continue + for conn_index, c in enumerate(this_query_connections): try: res = c.execute(q, query_id = run_id) except Exception as e: @@ -259,15 +263,35 @@ for query_index, q in enumerate(test_queries): e.message = run_id + ': ' + e.message raise - print(f'query\t{query_index}\t{run_id}\t{conn_index}\t{c.last_query.elapsed}') server_seconds += c.last_query.elapsed + print(f'query\t{query_index}\t{run_id}\t{conn_index}\t{c.last_query.elapsed}') + + # Be careful with the counter, after this line it's the next iteration + # already. + run += 1 + + # For very short queries we have a special mode where we run them for at + # least some time. The recommended lower bound of run time for "normal" + # queries is about 0.1 s, and we run them about 10 times, giving the + # time per query per server of about one second. Use this value as a + # reference for "short" queries. + if is_short[query_index]: + if server_seconds >= 1 * len(this_query_connections): + break + # Also limit the number of runs, so that we don't go crazy processing + # the results -- 'eqmed.sql' is really suboptimal. + if run >= 100: + break + else: + if run >= args.runs: + break client_seconds = time.perf_counter() - start_seconds print(f'client-time\t{query_index}\t{client_seconds}\t{server_seconds}') # Run drop queries drop_queries = substitute_parameters(drop_query_templates) -for conn_index, c in enumerate(connections): +for conn_index, c in enumerate(all_connections): for q in drop_queries: c.execute(q) print(f'drop\t{conn_index}\t{c.last_query.elapsed}\t{tsv_escape(q)}') From 00c924b0dd359686933f2d46c5b3d23955e7c32b Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 2 Sep 2020 06:31:53 +0300 Subject: [PATCH 16/68] whitespace --- docker/test/performance-comparison/compare.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index d8e3dc93442..aa3bb4cfc1a 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -546,8 +546,8 @@ create table test_time_changes engine File(TSV, 'report/test-time-changes.tsv') select test, count(*) queries, sum(left) as left, sum(right) as right, (right - left) / right average_time_change - from queries - group by test + from queries + group by test order by abs(average_time_change) desc ) ; From a321d6970c2ff77a3dd6a650a5ac696e2cff3e80 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 3 Sep 2020 04:42:25 +0300 Subject: [PATCH 17/68] adjust report and increase time --- docker/test/performance-comparison/compare.sh | 35 ++++++++----------- docker/test/performance-comparison/perf.py | 4 +-- docker/test/performance-comparison/report.py | 20 +++++------ 3 files changed, 25 insertions(+), 34 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index aa3bb4cfc1a..b98f89c6054 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -488,18 +488,11 @@ create view query_metric_stats as -- Main statistics for queries -- query time as reported in query log. create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv') as select - -- Comparison mode doesn't make sense for queries that complete - -- immediately (on the same order of time as noise). If query duration is - -- less that some threshold, we just skip it. If there is a significant - -- regression in such query, the time will exceed the threshold, and we - -- well process it normally and detect the regression. - right < $short_query_threshold as short, - - not short and abs(diff) > report_threshold and abs(diff) > stat_threshold as changed_fail, - not short and abs(diff) > report_threshold - 0.05 and abs(diff) > stat_threshold as changed_show, + abs(diff) > report_threshold and abs(diff) > stat_threshold as changed_fail, + abs(diff) > report_threshold - 0.05 and abs(diff) > stat_threshold as changed_show, - not short and not changed_fail and stat_threshold > report_threshold + 0.10 as unstable_fail, - not short and not changed_show and stat_threshold > report_threshold - 0.05 as unstable_show, + not changed_fail and stat_threshold > report_threshold + 0.10 as unstable_fail, + not changed_show and stat_threshold > report_threshold - 0.05 as unstable_show, left, right, diff, stat_threshold, if(report_threshold > 0, report_threshold, 0.10) as report_threshold, @@ -590,9 +583,9 @@ create table wall_clock_time_per_test engine Memory as select * create table test_time engine Memory as select test, sum(client) total_client_time, - maxIf(client, not short) query_max, - minIf(client, not short) query_min, - count(*) queries, sum(short) short_queries + max(client) query_max, + min(client) query_min, + count(*) queries from total_client_time_per_query full join queries using (test, query_index) group by test; @@ -600,7 +593,6 @@ create table test_times_report engine File(TSV, 'report/test-times.tsv') as select wall_clock_time_per_test.test, real, toDecimal64(total_client_time, 3), queries, - short_queries, toDecimal64(query_max, 3), toDecimal64(real / queries, 3) avg_real_per_query, toDecimal64(query_min, 3) @@ -641,17 +633,18 @@ create table unmarked_short_queries_report engine File(TSV, 'report/unmarked-short-queries.tsv') as select time, test, query_index, query_display_name from ( - select right time, test, query_index from queries where short + select right time, test, query_index from queries union all select time_median, test, query_index from partial_query_times - where time_median < $short_query_threshold ) times left join query_display_names on times.test = query_display_names.test and times.query_index = query_display_names.query_index - where (test, query_index) not in - (select * from file('analyze/marked-short-queries.tsv', TSV, - 'test text, query_index int')) + where + (test, query_index) not in + (select * from file('analyze/marked-short-queries.tsv', TSV, + 'test text, query_index int')) + and time < $short_query_threshold order by test, query_index ; @@ -660,7 +653,7 @@ create table unmarked_short_queries_report -- keep the table in old format so that we can analyze new and old data together create table queries_old_format engine File(TSVWithNamesAndTypes, 'queries.rep') - as select short, changed_fail, unstable_fail, left, right, diff, + as select 0 short, changed_fail, unstable_fail, left, right, diff, stat_threshold, test, query_display_name query from queries ; diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index fe8aff6b4cb..d96c475a43c 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -276,11 +276,11 @@ for query_index, q in enumerate(test_queries): # time per query per server of about one second. Use this value as a # reference for "short" queries. if is_short[query_index]: - if server_seconds >= 1 * len(this_query_connections): + if server_seconds >= 2 * len(this_query_connections): break # Also limit the number of runs, so that we don't go crazy processing # the results -- 'eqmed.sql' is really suboptimal. - if run >= 100: + if run >= 200: break else: if run >= args.runs: diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 4529718df51..0189c82935b 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -126,7 +126,6 @@ tr:nth-child(odd) td {{filter: brightness(90%);}} .test-times tr :nth-child(5), .test-times tr :nth-child(6), .test-times tr :nth-child(7), -.test-times tr :nth-child(8), .concurrent-benchmarks tr :nth-child(2), .concurrent-benchmarks tr :nth-child(3), .concurrent-benchmarks tr :nth-child(4), @@ -461,10 +460,9 @@ if args.report == 'main': 'Wall clock time, s', #1 'Total client time, s', #2 'Total queries', #3 - 'Ignored short queries', #4 - 'Longest query
(sum for all runs), s', #5 - 'Avg wall clock time
(sum for all runs), s', #6 - 'Shortest query
(sum for all runs), s', #7 + 'Longest query
(sum for all runs), s', #4 + 'Avg wall clock time
(sum for all runs), s', #5 + 'Shortest query
(sum for all runs), s', #6 ] text = tableStart('Test times') @@ -475,20 +473,20 @@ if args.report == 'main': attrs = ['' for c in columns] for r in rows: anchor = f'{currentTableAnchor()}.{r[0]}' - if float(r[6]) > 1.5 * total_runs: + if float(r[5]) > 1.5 * total_runs: # FIXME should be 15s max -- investigate parallel_insert slow_average_tests += 1 - attrs[6] = f'style="background: {color_bad}"' + attrs[5] = f'style="background: {color_bad}"' errors_explained.append([f'The test \'{r[0]}\' is too slow to run as a whole. Investigate whether the create and fill queries can be sped up']) else: - attrs[6] = '' + attrs[5] = '' - if float(r[5]) > allowed_single_run_time * total_runs: + if float(r[4]) > allowed_single_run_time * total_runs: slow_average_tests += 1 - attrs[5] = f'style="background: {color_bad}"' + attrs[4] = f'style="background: {color_bad}"' errors_explained.append([f'Some query of the test \'{r[0]}\' is too slow to run. See the all queries report']) else: - attrs[5] = '' + attrs[4] = '' text += tableRow(r, attrs, anchor) From 8689797efc4d749f4cda139818b1a94caccfe628 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 10 Sep 2020 19:57:26 +0300 Subject: [PATCH 18/68] Run only some queries in perf tests, not all combos --- docker/test/performance-comparison/compare.sh | 17 ++++++++++++++++- docker/test/performance-comparison/perf.py | 12 ++++++++++-- 2 files changed, 26 insertions(+), 3 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 364e9994ab7..0b678024765 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -121,7 +121,7 @@ function run_tests then # Use the explicitly set path to directory with test files. test_prefix="$CHPC_TEST_PATH" - elif [ "$PR_TO_TEST" = "0" ] + elif [ "$PR_TO_TEST" == "0" ] then # When testing commits from master, use the older test files. This # allows the tests to pass even when we add new functions and tests for @@ -155,6 +155,20 @@ function run_tests test_files=$(ls "$test_prefix"/*.xml) fi + # For PRs, test only a subset of queries, and run them less times. + # If the corresponding environment variables are already set, keep + # those values. + if [ "$PR_TO_TEST" == "0" ] + then + CHPC_TEST_RUNS=${CHPC_RUNS:-7} + CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-15} + else + CHPC_TEST_RUNS=${CHPC_RUNS:-13} + CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-0} + fi + export CHPC_TEST_RUNS + export CHPC_MAX_QUERIES + # Determine which concurrent benchmarks to run. For now, the only test # we run as a concurrent benchmark is 'website'. Run it as benchmark if we # are also going to run it as a normal test. @@ -187,6 +201,7 @@ function run_tests # the grep is to filter out set -x output and keep only time output { \ time "$script_dir/perf.py" --host localhost localhost --port 9001 9002 \ + --runs "$CHPC_RUNS" --max-queries "$CHPC_MAX_QUERIES" \ -- "$test" > "$test_name-raw.tsv" 2> "$test_name-err.log" ; \ } 2>&1 >/dev/null | grep -v ^+ >> "wall-clock-times.tsv" \ || echo "Test $test_name failed with error code $?" >> "$test_name-err.log" diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index e1476d9aeb4..64314c129b5 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -7,6 +7,7 @@ import clickhouse_driver import xml.etree.ElementTree as et import argparse import pprint +import random import re import string import time @@ -20,7 +21,8 @@ parser = argparse.ArgumentParser(description='Run performance test.') parser.add_argument('file', metavar='FILE', type=argparse.FileType('r', encoding='utf-8'), nargs=1, help='test description file') parser.add_argument('--host', nargs='*', default=['localhost'], help="Server hostname(s). Corresponds to '--port' options.") parser.add_argument('--port', nargs='*', default=[9000], help="Server port(s). Corresponds to '--host' options.") -parser.add_argument('--runs', type=int, default=int(os.environ.get('CHPC_RUNS', 13)), help='Number of query runs per server. Defaults to CHPC_RUNS environment variable.') +parser.add_argument('--runs', type=int, default=1, help='Number of query runs per server.') +parser.add_argument('--max-queries', type=int, default=None, help='Test no more than this number of queries, chosen at random.') parser.add_argument('--long', action='store_true', help='Do not skip the tests tagged as long.') parser.add_argument('--print-queries', action='store_true', help='Print test queries and exit.') parser.add_argument('--print-settings', action='store_true', help='Print test settings and exit.') @@ -189,8 +191,14 @@ for conn_index, c in enumerate(connections): c.execute(q) print(f'fill\t{conn_index}\t{c.last_query.elapsed}\t{tsv_escape(q)}') +# Run the queries in randomized order, but preserve their indexes as specified +# in the test XML. To avoid using too much time, limit the number of queries +# we run per test. +queries_to_run = random.sample(range(0, len(test_queries)), args.max_queries or len(test_queries)) + # Run test queries. -for query_index, q in enumerate(test_queries): +for query_index in queries_to_run: + q = test_queries[query_index] query_prefix = f'{test_name}.query{query_index}' # We have some crazy long queries (about 100kB), so trim them to a sane From a913be920117abad8e28039a69bacbe3d6307b6f Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 14 Sep 2020 19:36:14 +0800 Subject: [PATCH 19/68] add table function null --- src/TableFunctions/TableFunctionNull.cpp | 42 +++++++++++++++++++ src/TableFunctions/TableFunctionNull.h | 24 +++++++++++ src/TableFunctions/registerTableFunctions.cpp | 1 + src/TableFunctions/registerTableFunctions.h | 1 + 4 files changed, 68 insertions(+) create mode 100644 src/TableFunctions/TableFunctionNull.cpp create mode 100644 src/TableFunctions/TableFunctionNull.h diff --git a/src/TableFunctions/TableFunctionNull.cpp b/src/TableFunctions/TableFunctionNull.cpp new file mode 100644 index 00000000000..fe9c2d36d92 --- /dev/null +++ b/src/TableFunctions/TableFunctionNull.cpp @@ -0,0 +1,42 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include "registerTableFunctions.h" + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +StoragePtr TableFunctionNull::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +{ + if (const auto * function = ast_function->as()) + { + auto arguments = function->arguments->children; + + if (arguments.size() != 1) + throw Exception("Table function '" + getName() + "' requires 'structure'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + auto structure = arguments[0]->as().value.safeGet(); + ColumnsDescription columns = parseColumnsListFromString(structure, context); + + auto res = StorageNull::create(StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription()); + res->startup(); + return res; + } + throw Exception("Table function '" + getName() + "' requires 'structure'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); +} + +void registerTableFunctionNull(TableFunctionFactory & factory) +{ + factory.registerFunction(); +} +} diff --git a/src/TableFunctions/TableFunctionNull.h b/src/TableFunctions/TableFunctionNull.h new file mode 100644 index 00000000000..48617352b25 --- /dev/null +++ b/src/TableFunctions/TableFunctionNull.h @@ -0,0 +1,24 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/* null(structure) - creates a temporary null storage + * + * Used for testing purposes, for convenience writing tests and demos. + */ +class TableFunctionNull : public ITableFunction +{ +public: + static constexpr auto name = "null"; + std::string getName() const override { return name; } +private: + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; + const char * getStorageTypeName() const override { return "Null"; } +}; + +} diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 25a495a9185..09255c2bd74 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -11,6 +11,7 @@ void registerTableFunctions() registerTableFunctionMerge(factory); registerTableFunctionRemote(factory); registerTableFunctionNumbers(factory); + registerTableFunctionNull(factory); registerTableFunctionZeros(factory); registerTableFunctionFile(factory); registerTableFunctionURL(factory); diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index 8ff64a22fea..ab05187eeab 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -11,6 +11,7 @@ class TableFunctionFactory; void registerTableFunctionMerge(TableFunctionFactory & factory); void registerTableFunctionRemote(TableFunctionFactory & factory); void registerTableFunctionNumbers(TableFunctionFactory & factory); +void registerTableFunctionNull(TableFunctionFactory & factory); void registerTableFunctionZeros(TableFunctionFactory & factory); void registerTableFunctionFile(TableFunctionFactory & factory); void registerTableFunctionURL(TableFunctionFactory & factory); From 3795dfed144c93e57486c7d0ab5d370e9e8cc82b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Sep 2020 16:33:36 +0300 Subject: [PATCH 20/68] Remove one header --- base/common/CMakeLists.txt | 1 + base/common/StringRef.cpp | 13 +++++++++++++ base/common/StringRef.h | 10 ++-------- base/common/ya.make | 1 + src/Columns/ya.make | 2 ++ src/Common/ya.make | 1 + 6 files changed, 20 insertions(+), 8 deletions(-) create mode 100644 base/common/StringRef.cpp diff --git a/base/common/CMakeLists.txt b/base/common/CMakeLists.txt index 903168a0dd4..9e4462c821a 100644 --- a/base/common/CMakeLists.txt +++ b/base/common/CMakeLists.txt @@ -18,6 +18,7 @@ set (SRCS terminalColors.cpp errnoToString.cpp getResource.cpp + StringRef.cpp ) if (ENABLE_REPLXX) diff --git a/base/common/StringRef.cpp b/base/common/StringRef.cpp new file mode 100644 index 00000000000..87877360d83 --- /dev/null +++ b/base/common/StringRef.cpp @@ -0,0 +1,13 @@ +#include + +#include "StringRef.h" + + +std::ostream & operator<<(std::ostream & os, const StringRef & str) +{ + if (str.data) + os.write(str.data, str.size); + + return os; +} + diff --git a/base/common/StringRef.h b/base/common/StringRef.h index 410e13ba7d8..05d4eda7656 100644 --- a/base/common/StringRef.h +++ b/base/common/StringRef.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include @@ -322,10 +322,4 @@ inline bool operator==(StringRef lhs, const char * rhs) return true; } -inline std::ostream & operator<<(std::ostream & os, const StringRef & str) -{ - if (str.data) - os.write(str.data, str.size); - - return os; -} +std::ostream & operator<<(std::ostream & os, const StringRef & str); diff --git a/base/common/ya.make b/base/common/ya.make index 2bd08afbf3a..cbb6b5f64ac 100644 --- a/base/common/ya.make +++ b/base/common/ya.make @@ -53,6 +53,7 @@ SRCS( setTerminalEcho.cpp shift10.cpp sleep.cpp + StringRef.cpp terminalColors.cpp ) diff --git a/src/Columns/ya.make b/src/Columns/ya.make index 910c479c2a9..78c0e1b992d 100644 --- a/src/Columns/ya.make +++ b/src/Columns/ya.make @@ -2,6 +2,8 @@ LIBRARY() ADDINCL( + contrib/libs/icu/common + contrib/libs/icu/i18n contrib/libs/pdqsort ) diff --git a/src/Common/ya.make b/src/Common/ya.make index d9a7a2ce4de..2478fa0c9ce 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -86,6 +86,7 @@ SRCS( StatusFile.cpp StatusInfo.cpp Stopwatch.cpp + StringRef.cpp StringUtils/StringUtils.cpp StudentTTest.cpp SymbolIndex.cpp From 17a04cd62b2ad2adc6adfd3afbcb9a7750bcc5f0 Mon Sep 17 00:00:00 2001 From: yulu86 Date: Mon, 14 Sep 2020 22:44:56 +0800 Subject: [PATCH 21/68] Optimize Chinese tutorial to make it more human readable --- docs/zh/getting-started/tutorial.md | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/zh/getting-started/tutorial.md b/docs/zh/getting-started/tutorial.md index 43c7ed0ec59..2a82911cce4 100644 --- a/docs/zh/getting-started/tutorial.md +++ b/docs/zh/getting-started/tutorial.md @@ -80,7 +80,7 @@ clickhouse-client --query='INSERT INTO table FORMAT TabSeparated' < data.tsv ## 导入示例数据集 {#import-sample-dataset} -现在是时候用一些示例数据填充我们的ClickHouse服务器。 在本教程中,我们将使用Yandex的匿名数据。Metrica,在成为开源之前以生产方式运行ClickHouse的第一个服务(更多关于这一点 [历史科](../introduction/history.md)). 有 [多种导入Yandex的方式。梅里卡数据集](example-datasets/metrica.md),为了本教程,我们将使用最现实的一个。 +现在是时候用一些示例数据填充我们的ClickHouse服务端。 在本教程中,我们将使用Yandex.Metrica的匿名数据,它是在ClickHouse成为开源之前作为生产环境运行的第一个服务(关于这一点的更多内容请参阅[ClickHouse历史](../introduction/history.md))。有 [多种导入Yandex.Metrica数据集的的方法](example-datasets/metrica.md),为了本教程,我们将使用最现实的一个。 ### 下载并提取表数据 {#download-and-extract-table-data} @@ -93,22 +93,22 @@ curl https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz | unx ### 创建表 {#create-tables} -与大多数数据库管理系统一样,ClickHouse在逻辑上将表分组为 “databases”. 有一个 `default` 数据库,但我们将创建一个名为新的 `tutorial`: +与大多数数据库管理系统一样,ClickHouse在逻辑上将表分组为数据库。包含一个 `default` 数据库,但我们将创建一个新的数据库 `tutorial`: ``` bash clickhouse-client --query "CREATE DATABASE IF NOT EXISTS tutorial" ``` -与数据库相比,创建表的语法要复杂得多(请参阅 [参考资料](../sql-reference/statements/create.md). 一般 `CREATE TABLE` 声明必须指定三个关键的事情: +与创建数据库相比,创建表的语法要复杂得多(请参阅 [参考资料](../sql-reference/statements/create.md). 一般 `CREATE TABLE` 声明必须指定三个关键的事情: 1. 要创建的表的名称。 -2. Table schema, i.e. list of columns and their [数据类型](../sql-reference/data-types/index.md). -3. [表引擎](../engines/table-engines/index.md) 及其设置,这决定了如何物理执行对此表的查询的所有细节。 +2. 表结构,例如:列名和对应的[数据类型](../sql-reference/data-types/index.md)。 +3. [表引擎](../engines/table-engines/index.md) 及其设置,这决定了对此表的查询操作是如何在物理层面执行的所有细节。 -YandexMetrica是一个网络分析服务,样本数据集不包括其全部功能,因此只有两个表可以创建: +Yandex.Metrica是一个网络分析服务,样本数据集不包括其全部功能,因此只有两个表可以创建: -- `hits` 是一个表格,其中包含所有用户在服务所涵盖的所有网站上完成的每个操作。 -- `visits` 是一个包含预先构建的会话而不是单个操作的表。 +- `hits` 表包含所有用户在服务所涵盖的所有网站上完成的每个操作。 +- `visits` 表包含预先构建的会话,而不是单个操作。 让我们看看并执行这些表的实际创建表查询: @@ -453,9 +453,9 @@ SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192 ``` -您可以使用以下交互模式执行这些查询 `clickhouse-client` (只需在终端中启动它,而不需要提前指定查询)或尝试一些 [替代接口](../interfaces/index.md) 如果你愿意的话 +您可以使用`clickhouse-client`的交互模式执行这些查询(只需在终端中启动它,而不需要提前指定查询)。或者如果你愿意,可以尝试一些[替代接口](../interfaces/index.md)。 -正如我们所看到的, `hits_v1` 使用 [基本MergeTree引擎](../engines/table-engines/mergetree-family/mergetree.md),而 `visits_v1` 使用 [崩溃](../engines/table-engines/mergetree-family/collapsingmergetree.md) 变体。 +正如我们所看到的, `hits_v1` 使用 [基本的MergeTree引擎](../engines/table-engines/mergetree-family/mergetree.md),而 `visits_v1` 使用 [折叠树](../engines/table-engines/mergetree-family/collapsingmergetree.md) 变体。 ### 导入数据 {#import-data} From ab6bc1ed59449a200e1ea9c0fe96beb7a3fc4fd7 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Mon, 14 Sep 2020 20:25:17 +0300 Subject: [PATCH 22/68] Update compare.sh --- docker/test/performance-comparison/compare.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 0b678024765..db4939d529d 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -198,12 +198,13 @@ function run_tests echo test "$test_name" TIMEFORMAT=$(printf "$test_name\t%%3R\t%%3U\t%%3S\n") - # the grep is to filter out set -x output and keep only time output + # The grep is to filter out set -x output and keep only time output. + # The '2>&1 >/dev/null' redirects stderr to stdout, and discards stdout. { \ time "$script_dir/perf.py" --host localhost localhost --port 9001 9002 \ --runs "$CHPC_RUNS" --max-queries "$CHPC_MAX_QUERIES" \ -- "$test" > "$test_name-raw.tsv" 2> "$test_name-err.log" ; \ - } 2>&1 >/dev/null | grep -v ^+ >> "wall-clock-times.tsv" \ + } 2>&1 >/dev/null | tee >(grep -v ^+ >> "wall-clock-times.tsv") \ || echo "Test $test_name failed with error code $?" >> "$test_name-err.log" done From 03346a0a3024288850cd24e91884d5e0cb5889fe Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 15 Sep 2020 17:35:38 +0800 Subject: [PATCH 23/68] add performance test --- tests/performance/table_function_null.xml | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 tests/performance/table_function_null.xml diff --git a/tests/performance/table_function_null.xml b/tests/performance/table_function_null.xml new file mode 100644 index 00000000000..9313619d89a --- /dev/null +++ b/tests/performance/table_function_null.xml @@ -0,0 +1,3 @@ + + INSERT INTO function null('number UInt64') SELECT * FROM numbers_mt(1000000000); + From 24dd33d5cbd6814c15d1ed7fc487988c46d66b16 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Tue, 15 Sep 2020 13:44:21 +0300 Subject: [PATCH 24/68] Update compare.sh --- docker/test/performance-comparison/compare.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index db4939d529d..16aff19bc19 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -160,13 +160,13 @@ function run_tests # those values. if [ "$PR_TO_TEST" == "0" ] then - CHPC_TEST_RUNS=${CHPC_RUNS:-7} - CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-15} + CHPC_RUNS=${CHPC_RUNS:-7} + CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-100} else - CHPC_TEST_RUNS=${CHPC_RUNS:-13} + CHPC_RUNS=${CHPC_RUNS:-13} CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-0} fi - export CHPC_TEST_RUNS + export CHPC_RUNS export CHPC_MAX_QUERIES # Determine which concurrent benchmarks to run. For now, the only test From c5c5faf8ee46228288a1efc70baf794aac27b363 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 16 Sep 2020 12:06:42 +0300 Subject: [PATCH 25/68] fixup --- docker/test/performance-comparison/compare.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index b768d26d478..00cafadb502 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -160,11 +160,11 @@ function run_tests # those values. if [ "$PR_TO_TEST" == "0" ] then - CHPC_RUNS=${CHPC_RUNS:-7} - CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-100} - else CHPC_RUNS=${CHPC_RUNS:-13} CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-0} + else + CHPC_RUNS=${CHPC_RUNS:-7} + CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-100} fi export CHPC_RUNS export CHPC_MAX_QUERIES From 3b8ca1f26240777643b09488394bf0f4d50acbaf Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 16 Sep 2020 12:16:07 +0300 Subject: [PATCH 26/68] Update compare.sh --- docker/test/performance-comparison/compare.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 00cafadb502..18e5bea4db5 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -164,7 +164,7 @@ function run_tests CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-0} else CHPC_RUNS=${CHPC_RUNS:-7} - CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-100} + CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-20} fi export CHPC_RUNS export CHPC_MAX_QUERIES From d55e3cd21458a1d635041546b691c161721e3d40 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 16 Sep 2020 14:39:44 +0300 Subject: [PATCH 27/68] Update perf.py --- docker/test/performance-comparison/perf.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 3bd997d030b..d54d6444ee0 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -280,7 +280,7 @@ for query_index, q in enumerate(test_queries): break # Also limit the number of runs, so that we don't go crazy processing # the results -- 'eqmed.sql' is really suboptimal. - if run >= 200: + if run >= 500: break else: if run >= args.runs: From d99a011d07bee19e20d2626de02b0a4bd5d6a3fc Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 16 Sep 2020 15:34:38 +0300 Subject: [PATCH 28/68] Fix removing a live view after watching and after timeout has passed. --- .../LiveView/TemporaryLiveViewCleaner.cpp | 33 +++++++++++++------ 1 file changed, 23 insertions(+), 10 deletions(-) diff --git a/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp b/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp index 0f7c1039d72..1159a93d2ef 100644 --- a/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp +++ b/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp @@ -69,20 +69,29 @@ TemporaryLiveViewCleaner::~TemporaryLiveViewCleaner() void TemporaryLiveViewCleaner::addView(const std::shared_ptr & view) { - if (!view->isTemporary()) + if (!view->isTemporary() || background_thread_should_exit) return; auto current_time = std::chrono::system_clock::now(); auto time_of_next_check = current_time + view->getTimeout(); std::lock_guard lock{mutex}; + if (background_thread_should_exit) + return; + + /// If views.empty() the background thread isn't running or it's going to stop right now. + bool background_thread_is_running = !views.empty(); /// Keep the vector `views` sorted by time of next check. StorageAndTimeOfCheck storage_and_time_of_check{view, time_of_next_check}; views.insert(std::upper_bound(views.begin(), views.end(), storage_and_time_of_check), storage_and_time_of_check); - if (!background_thread.joinable()) + if (!background_thread_is_running) + { + if (background_thread.joinable()) + background_thread.join(); background_thread = ThreadFromGlobalPool{&TemporaryLiveViewCleaner::backgroundThreadFunc, this}; + } background_thread_wake_up.notify_one(); } @@ -95,7 +104,7 @@ void TemporaryLiveViewCleaner::backgroundThreadFunc() { background_thread_wake_up.wait_until(lock, views.front().time_of_check); if (background_thread_should_exit) - return; + break; auto current_time = std::chrono::system_clock::now(); std::vector storages_to_drop; @@ -112,18 +121,22 @@ void TemporaryLiveViewCleaner::backgroundThreadFunc() continue; } - ++it; - if (current_time < time_of_check) break; /// It's not the time to check it yet. + auto storage_id = storage->getStorageID(); + if (!storage->hasUsers() && DatabaseCatalog::instance().getDependencies(storage_id).empty()) + { + /// No users and no dependencies so we can remove the storage. + storages_to_drop.emplace_back(storage_id); + it = views.erase(it); + continue; + } + + /// Calculate time of the next check. time_of_check = current_time + storage->getTimeout(); - auto storage_id = storage->getStorageID(); - if (storage->hasUsers() || !DatabaseCatalog::instance().getDependencies(storage_id).empty()) - continue; - - storages_to_drop.emplace_back(storage_id); + ++it; } lock.unlock(); From dda88eebd11298237894b253523c7d070e53162f Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 16 Sep 2020 18:35:27 +0300 Subject: [PATCH 29/68] test again --- docker/test/performance-comparison/compare.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 18e5bea4db5..8d7947b46a5 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -63,7 +63,7 @@ function configure # Make copies of the original db for both servers. Use hardlinks instead # of copying to save space. Before that, remove preprocessed configs and # system tables, because sharing them between servers with hardlinks may - # lead to weird effects. + # lead to weird effects. rm -r left/db ||: rm -r right/db ||: rm -r db0/preprocessed_configs ||: From 7d046b24e653a5d19737b422a25456ae8b50918d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 16 Sep 2020 23:56:16 +0300 Subject: [PATCH 30/68] Improve error message for INSERT via clickhouse-client With '\n...' after the query [1] clickhouse-client prefer data from the INSERT over from stdin, and produce very tricky message: Code: 27. DB::Exception: Cannot parse input: expected '\n' before: ' ': (at row 1) Well for TSV it is ok, but for RowBinary: Code: 33. DB::Exception: Cannot read all data. Bytes read: 1. Bytes expected: 4. So improve error message by adding the source of data for INSERT. [1]: clickhouse-client -q "INSERT INTO data FORMAT TSV\n " <<<2 --- programs/client/Client.cpp | 23 +++++++++++++++++-- ...ickhouse_client_INSERT_exception.reference | 2 ++ ...1501_clickhouse_client_INSERT_exception.sh | 15 ++++++++++++ 3 files changed, 38 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01501_clickhouse_client_INSERT_exception.reference create mode 100755 tests/queries/0_stateless/01501_clickhouse_client_INSERT_exception.sh diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 0c2aca2b3c8..acb5108b60d 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1502,7 +1502,18 @@ 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); + try + { + sendDataFrom(data_in, sample, columns_description); + } + catch (Exception & e) + { + /// The following query will use data from input + // "INSERT INTO data FORMAT TSV\n " < data.csv + // And may be pretty hard to debug, so add information about data source to make it easier. + e.addMessage("data for INSERT was parsed from query"); + throw; + } // 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(); @@ -1510,7 +1521,15 @@ private: else if (!is_interactive) { /// Send data read from stdin. - sendDataFrom(std_in, sample, columns_description); + try + { + sendDataFrom(std_in, sample, columns_description); + } + catch (Exception & e) + { + e.addMessage("data for INSERT was parsed from stdin"); + throw; + } } else throw Exception("No data to insert", ErrorCodes::NO_DATA_TO_INSERT); diff --git a/tests/queries/0_stateless/01501_clickhouse_client_INSERT_exception.reference b/tests/queries/0_stateless/01501_clickhouse_client_INSERT_exception.reference new file mode 100644 index 00000000000..b258dc3d8a7 --- /dev/null +++ b/tests/queries/0_stateless/01501_clickhouse_client_INSERT_exception.reference @@ -0,0 +1,2 @@ +1 +10 diff --git a/tests/queries/0_stateless/01501_clickhouse_client_INSERT_exception.sh b/tests/queries/0_stateless/01501_clickhouse_client_INSERT_exception.sh new file mode 100755 index 00000000000..5f4f9836d35 --- /dev/null +++ b/tests/queries/0_stateless/01501_clickhouse_client_INSERT_exception.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS data" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE data (key Int) Engine=Memory()" +${CLICKHOUSE_CLIENT} --input_format_parallel_parsing=0 -q "INSERT INTO data SELECT key FROM input('key Int') FORMAT TSV" <<<10 +# with '\n...' after the query clickhouse-client prefer data from the query over data from stdin, and produce very tricky message: +# Code: 27. DB::Exception: Cannot parse input: expected '\n' before: ' ': (at row 1) +# well for TSV it is ok, but for RowBinary: +# Code: 33. DB::Exception: Cannot read all data. Bytes read: 1. Bytes expected: 4. +# so check that the exception message contain the data source. +${CLICKHOUSE_CLIENT} --input_format_parallel_parsing=0 -q "INSERT INTO data FORMAT TSV " <<<2 |& grep -F -c 'data for INSERT was parsed from query' +${CLICKHOUSE_CLIENT} -q "SELECT * FROM data" From 13088d9befdd0e2d91764a6ba89c80bb88227138 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 17 Sep 2020 08:04:50 +0300 Subject: [PATCH 31/68] Fix 00900_parquet_load (update exception message on INSERT failures) --- .../0_stateless/00900_parquet_load.reference | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/00900_parquet_load.reference b/tests/queries/0_stateless/00900_parquet_load.reference index f49b74e4c3c..6cd2b1cf462 100644 --- a/tests/queries/0_stateless/00900_parquet_load.reference +++ b/tests/queries/0_stateless/00900_parquet_load.reference @@ -39,7 +39,7 @@ 23.0 24.0 === Try load data from datapage_v2.snappy.parquet -Code: 33. DB::Ex---tion: Error while reading Parquet data: IOError: Not yet implemented: Unsupported encoding. +Code: 33. DB::Ex---tion: Error while reading Parquet data: IOError: Not yet implemented: Unsupported encoding.: data for INSERT was parsed from stdin === Try load data from fixed_length_decimal_1.parquet 1.0 @@ -168,22 +168,22 @@ Code: 33. DB::Ex---tion: Error while reading Parquet data: IOError: Not yet impl 23 UNITED KINGDOM 3 eans boost carefully special requests. accounts are. carefull 24 UNITED STATES 1 y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be === Try load data from nested_lists.snappy.parquet -Code: 8. DB::Ex---tion: Column "element" is not presented in input data +Code: 8. DB::Ex---tion: Column "element" is not presented in input data: data for INSERT was parsed from stdin === Try load data from nested_maps.snappy.parquet -Code: 33. DB::Ex---tion: Error while reading Parquet data: NotImplemented: Reading lists of structs from Parquet files not yet supported: key_value: list not null> not null>> not null> not null +Code: 33. DB::Ex---tion: Error while reading Parquet data: NotImplemented: Reading lists of structs from Parquet files not yet supported: key_value: list not null> not null>> not null> not null: data for INSERT was parsed from stdin === Try load data from nonnullable.impala.parquet -Code: 8. DB::Ex---tion: Column "element" is not presented in input data +Code: 8. DB::Ex---tion: Column "element" is not presented in input data: data for INSERT was parsed from stdin === Try load data from nullable.impala.parquet -Code: 8. DB::Ex---tion: Column "element" is not presented in input data +Code: 8. DB::Ex---tion: Column "element" is not presented in input data: data for INSERT was parsed from stdin === Try load data from nulls.snappy.parquet -Code: 8. DB::Ex---tion: Column "b_c_int" is not presented in input data +Code: 8. DB::Ex---tion: Column "b_c_int" is not presented in input data: data for INSERT was parsed from stdin === Try load data from repeated_no_annotation.parquet -Code: 8. DB::Ex---tion: Column "number" is not presented in input data +Code: 8. DB::Ex---tion: Column "number" is not presented in input data: data for INSERT was parsed from stdin === Try load data from userdata1.parquet 1454486129 1 Amanda Jordan ajordan0@com.com Female 1.197.201.2 6759521864920116 Indonesia 3/8/1971 49756.53 Internal Auditor 1E+02 From b41421cb1cdc6c3f86ef46f27521612621eef3a3 Mon Sep 17 00:00:00 2001 From: roman Date: Fri, 28 Aug 2020 15:50:25 +0100 Subject: [PATCH 32/68] [settings]: introduce new query complexity settings for leaf-nodes The new setting should allow to control query complexity on leaf nodes excluding the final merging stage on the root-node. For example, distributed query that reads 1k rows from 5 shards will breach the `max_rows_to_read=5000`, while effectively every shard reads only 1k rows. With setting `max_rows_to_read_leaf=1500` this limit won't be reached and query will succeed since every shard reads not more that ~1k rows. --- .../operations/settings/query-complexity.md | 25 ++++++++++++++++ .../operations/settings/query-complexity.md | 26 +++++++++++++++++ src/Core/Settings.h | 4 +++ src/Interpreters/InterpreterSelectQuery.cpp | 9 ++++-- src/Processors/Pipe.cpp | 9 ++++++ src/Processors/Pipe.h | 1 + .../QueryPlan/ReadFromStorageStep.cpp | 12 ++++++++ .../QueryPlan/ReadFromStorageStep.h | 2 ++ .../Sources/SourceFromInputStream.h | 1 + src/Processors/Sources/SourceWithProgress.cpp | 6 ++++ src/Processors/Sources/SourceWithProgress.h | 5 ++++ src/Storages/IStorage.cpp | 3 +- src/Storages/IStorage.h | 1 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 18 +++++++----- ...hard_leaf_max_rows_bytes_to_read.reference | 6 ++++ ...1455_shard_leaf_max_rows_bytes_to_read.sql | 29 +++++++++++++++++++ 16 files changed, 147 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/01455_shard_leaf_max_rows_bytes_to_read.reference create mode 100755 tests/queries/0_stateless/01455_shard_leaf_max_rows_bytes_to_read.sql diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index 0486392d259..f803e694eb7 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -60,6 +60,31 @@ A maximum number of bytes (uncompressed data) that can be read from a table when What to do when the volume of data read exceeds one of the limits: ‘throw’ or ‘break’. By default, throw. +## max\_rows\_to\_read_leaf {#max-rows-to-read-leaf} + +The following restrictions can be checked on each block (instead of on each row). That is, the restrictions can be broken a little. + +A maximum number of rows that can be read from a local table on a leaf node when running a distributed query. While +distributed queries can issue a multiple sub-queries to each shard (leaf) - this limit will be checked only on the read +stage on the leaf nodes and ignored on results merging stage on the root node. For example, cluster consists of 2 shards +and each shard contains a table with 100 rows. Then distributed query which suppose to read all the data from both +tables with setting `max_rows_to_read=150` will fail as in total it will be 200 rows. While query +with `max_rows_to_read_leaf=150` will succeed since leaf nodes will read 100 rows at max. + +## max\_bytes\_to\_read_leaf {#max-bytes-to-read-leaf} + +A maximum number of bytes (uncompressed data) that can be read from a local table on a leaf node when running +a distributed query. While distributed queries can issue a multiple sub-queries to each shard (leaf) - this limit will +be checked only on the read stage on the leaf nodes and ignored on results merging stage on the root node. +For example, cluster consists of 2 shards and each shard contains a table with 100 bytes of data. +Then distributed query which suppose to read all the data from both tables with setting `max_bytes_to_read=150` will fail +as in total it will be 200 bytes. While query with `max_bytes_to_read_leaf=150` will succeed since leaf nodes will read +100 bytes at max. + +## read\_overflow\_mode_leaf {#read-overflow-mode-leaf} + +What to do when the volume of data read exceeds one of the leaf limits: ‘throw’ or ‘break’. By default, throw. + ## max\_rows\_to\_group\_by {#settings-max-rows-to-group-by} A maximum number of unique keys received from aggregation. This setting lets you limit memory consumption when aggregating. diff --git a/docs/ru/operations/settings/query-complexity.md b/docs/ru/operations/settings/query-complexity.md index 74c99968bc0..d228732acdf 100644 --- a/docs/ru/operations/settings/query-complexity.md +++ b/docs/ru/operations/settings/query-complexity.md @@ -56,6 +56,32 @@ Что делать, когда количество прочитанных данных превысило одно из ограничений: throw или break. По умолчанию: throw. +## max\_rows\_to\_read_leaf {#max-rows-to-read-leaf} + +Следующие ограничения могут проверяться на каждый блок (а не на каждую строку). То есть, ограничения могут быть немного нарушены. + +Максимальное количество строчек, которое можно прочитать из таблицы на удалённом сервере при выполнении +распределенного запроса. Распределенные запросы могут создавать несколько подзапросов к каждому из шардов в кластере и +тогда этот лимит будет применен при выполнении чтения на удаленных серверах (включая и сервер-инициатор) и проигнорирован +на сервере-инициаторе запроса во время обьединения полученных результатов. Например, кластер состоит из 2 шард и каждый +из них хранит таблицу с 100 строк. Тогда распределнный запрос для получения всех данных из этих таблиц и установленной +настройкой `max_rows_to_read=150` выбросит исключение, т.к. в общем он прочитает 200 строк. Но запрос +с настройкой `max_rows_to_read_leaf=150` завершится успешно, потому что каждый из шардов прочитает максимум 100 строк. + +## max\_bytes\_to\_read_leaf {#max-bytes-to-read-leaf} + +Максимальное количество байт (несжатых данных), которое можно прочитать из таблицы на удалённом сервере при +выполнении распределенного запроса. Распределенные запросы могут создавать несколько подзапросов к каждому из шардов в +кластере и тогда этот лимит будет применен при выполнении чтения на удаленных серверах (включая и сервер-инициатор) +и проигнорирован на сервере-инициаторе запроса во время обьединения полученных результатов. Например, кластер состоит +из 2 шард и каждый из них хранит таблицу со 100 байтами. Тогда распределнный запрос для получения всех данных из этих таблиц +и установленной настройкой `max_bytes_to_read=150` выбросит исключение, т.к. в общем он прочитает 200 байт. Но запрос +с настройкой `max_bytes_to_read_leaf=150` завершится успешно, потому что каждый из шардов прочитает максимум 100 байт. + +## read\_overflow\_mode_leaf {#read-overflow-mode-leaf} + +Что делать, когда количество прочитанных данных на удаленном сервере превысило одно из ограничений: throw или break. По умолчанию: throw. + ## max\_rows\_to\_group\_by {#settings-max-rows-to-group-by} Максимальное количество уникальных ключей, получаемых в процессе агрегации. Позволяет ограничить потребление оперативки при агрегации. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b39c223a5e9..bf1a44670bc 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -232,6 +232,10 @@ class IColumn; M(UInt64, max_bytes_to_read, 0, "Limit on read bytes (after decompression) from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.", 0) \ M(OverflowMode, read_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ \ + M(UInt64, max_rows_to_read_leaf, 0, "Limit on read rows on the leaf nodes for distributed queries. Limit is applied for local reads only excluding the final merge stage on the root node.", 0) \ + M(UInt64, max_bytes_to_read_leaf, 0, "Limit on read bytes (after decompression) on the leaf nodes for distributed queries. Limit is applied for local reads only excluding the final merge stage on the root node.", 0) \ + M(OverflowMode, read_overflow_mode_leaf, OverflowMode::THROW, "What to do when the leaf limit is exceeded.", 0) \ + \ M(UInt64, max_rows_to_group_by, 0, "", 0) \ M(OverflowModeGroupBy, group_by_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ M(UInt64, max_bytes_before_external_group_by, 0, "", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 887f4795bcb..3cbbdb576b3 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1441,16 +1441,21 @@ void InterpreterSelectQuery::executeFetchColumns( } StreamLocalLimits limits; + SizeLimits leaf_limits; std::shared_ptr quota; + /// Set the limits and quota for reading data, the speed and time of the query. - if (!options.ignore_limits) + if (!options.ignore_limits) { limits = getLimitsForStorage(settings, options); + leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, settings.max_bytes_to_read_leaf, + settings.read_overflow_mode_leaf); + } if (!options.ignore_quota && (options.to_stage == QueryProcessingStage::Complete)) quota = context->getQuota(); - storage->read(query_plan, table_lock, metadata_snapshot, limits, std::move(quota), + storage->read(query_plan, table_lock, metadata_snapshot, limits, leaf_limits, std::move(quota), required_columns, query_info, context, processing_stage, max_block_size, max_streams); } else diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index 90a8a65ff25..9e9c9cab385 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -788,6 +788,15 @@ void Pipe::setLimits(const StreamLocalLimits & limits) } } +void Pipe::setLeafLimits(const SizeLimits & leaf_limits) +{ + for (auto & processor : processors) + { + if (auto * source_with_progress = dynamic_cast(processor.get())) + source_with_progress->setLeafLimits(leaf_limits); + } +} + void Pipe::setQuota(const std::shared_ptr & quota) { for (auto & processor : processors) diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 4adb529bb1e..f674663154d 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -97,6 +97,7 @@ public: /// Specify quotas and limits for every ISourceWithProgress. void setLimits(const StreamLocalLimits & limits); + void setLeafLimits(const SizeLimits & leaf_limits); void setQuota(const std::shared_ptr & quota); /// Do not allow to change the table while the processors of pipe are alive. diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.cpp b/src/Processors/QueryPlan/ReadFromStorageStep.cpp index 2f305e7220c..b085c177ad4 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromStorageStep.cpp @@ -15,6 +15,7 @@ ReadFromStorageStep::ReadFromStorageStep( TableLockHolder table_lock_, StorageMetadataPtr metadata_snapshot_, StreamLocalLimits & limits_, + SizeLimits & leaf_limits_, std::shared_ptr quota_, StoragePtr storage_, const Names & required_columns_, @@ -26,6 +27,7 @@ ReadFromStorageStep::ReadFromStorageStep( : table_lock(std::move(table_lock_)) , metadata_snapshot(std::move(metadata_snapshot_)) , limits(limits_) + , leaf_limits(leaf_limits_) , quota(std::move(quota_)) , storage(std::move(storage_)) , required_columns(required_columns_) @@ -86,6 +88,16 @@ ReadFromStorageStep::ReadFromStorageStep( pipe.setLimits(limits); + /** + * Leaf size limits should be applied only for local processing of distributed queries. + * Such limits allow to control the read stage on leaf nodes and exclude the merging stage. + * Consider the case when distributed query needs to read from multiple shards. Then leaf + * limits will be applied on the shards only (including the root node) but will be ignored + * on the results merging stage. + */ + if (!storage->isRemote()) + pipe.setLeafLimits(leaf_limits); + if (quota) pipe.setQuota(quota); diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.h b/src/Processors/QueryPlan/ReadFromStorageStep.h index 9c2b9e56450..98cde63a863 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.h +++ b/src/Processors/QueryPlan/ReadFromStorageStep.h @@ -26,6 +26,7 @@ public: TableLockHolder table_lock, StorageMetadataPtr metadata_snapshot, StreamLocalLimits & limits, + SizeLimits & leaf_limits, std::shared_ptr quota, StoragePtr storage, const Names & required_columns, @@ -47,6 +48,7 @@ private: TableLockHolder table_lock; StorageMetadataPtr metadata_snapshot; StreamLocalLimits limits; + SizeLimits leaf_limits; std::shared_ptr quota; StoragePtr storage; diff --git a/src/Processors/Sources/SourceFromInputStream.h b/src/Processors/Sources/SourceFromInputStream.h index 630c712daef..2e8cf007623 100644 --- a/src/Processors/Sources/SourceFromInputStream.h +++ b/src/Processors/Sources/SourceFromInputStream.h @@ -33,6 +33,7 @@ public: /// Implementation for methods from ISourceWithProgress. void setLimits(const StreamLocalLimits & limits_) final { stream->setLimits(limits_); } + void setLeafLimits(const SizeLimits &) final { } void setQuota(const std::shared_ptr & quota_) final { stream->setQuota(quota_); } void setProcessListElement(QueryStatus * elem) final { stream->setProcessListElement(elem); } void setProgressCallback(const ProgressCallback & callback) final { stream->setProgressCallback(callback); } diff --git a/src/Processors/Sources/SourceWithProgress.cpp b/src/Processors/Sources/SourceWithProgress.cpp index d6972f99369..e3dd614eece 100644 --- a/src/Processors/Sources/SourceWithProgress.cpp +++ b/src/Processors/Sources/SourceWithProgress.cpp @@ -93,6 +93,12 @@ void SourceWithProgress::progress(const Progress & value) } } + if (!leaf_limits.check(rows_to_check_limit, progress.read_bytes,"rows or bytes to read on leaf node", + ErrorCodes::TOO_MANY_ROWS, ErrorCodes::TOO_MANY_BYTES)) + { + cancel(); + } + size_t total_rows = progress.total_rows_to_read; constexpr UInt64 profile_events_update_period_microseconds = 10 * 1000; // 10 milliseconds diff --git a/src/Processors/Sources/SourceWithProgress.h b/src/Processors/Sources/SourceWithProgress.h index fdab345548b..3aa7a81f418 100644 --- a/src/Processors/Sources/SourceWithProgress.h +++ b/src/Processors/Sources/SourceWithProgress.h @@ -17,6 +17,9 @@ public: /// Set limitations that checked on each chunk. virtual void setLimits(const StreamLocalLimits & limits_) = 0; + /// Set limitations that checked on each chunk for distributed queries on leaf nodes. + virtual void setLeafLimits(const SizeLimits & leaf_limits_) = 0; + /// Set the quota. If you set a quota on the amount of raw data, /// then you should also set mode = LIMITS_TOTAL to LocalLimits with setLimits. virtual void setQuota(const std::shared_ptr & quota_) = 0; @@ -46,6 +49,7 @@ public: SourceWithProgress(Block header, bool enable_auto_progress); void setLimits(const StreamLocalLimits & limits_) final { limits = limits_; } + void setLeafLimits(const SizeLimits & leaf_limits_) final {leaf_limits = leaf_limits_; } void setQuota(const std::shared_ptr & quota_) final { quota = quota_; } void setProcessListElement(QueryStatus * elem) final { process_list_elem = elem; } void setProgressCallback(const ProgressCallback & callback) final { progress_callback = callback; } @@ -59,6 +63,7 @@ protected: private: StreamLocalLimits limits; + SizeLimits leaf_limits; std::shared_ptr quota; ProgressCallback progress_callback; QueryStatus * process_list_elem = nullptr; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 0711d32d802..50b36ced19c 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -97,6 +97,7 @@ void IStorage::read( TableLockHolder table_lock, StorageMetadataPtr metadata_snapshot, StreamLocalLimits & limits, + SizeLimits & leaf_limits, std::shared_ptr quota, const Names & column_names, const SelectQueryInfo & query_info, @@ -106,7 +107,7 @@ void IStorage::read( unsigned num_streams) { auto read_step = std::make_unique( - std::move(table_lock), std::move(metadata_snapshot), limits, std::move(quota), shared_from_this(), + std::move(table_lock), std::move(metadata_snapshot), limits, leaf_limits, std::move(quota), shared_from_this(), column_names, query_info, std::move(context), processed_stage, max_block_size, num_streams); read_step->setStepDescription("Read from " + getName()); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index dc7c684d5b4..dbd18c9558e 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -288,6 +288,7 @@ public: TableLockHolder table_lock, StorageMetadataPtr metadata_snapshot, StreamLocalLimits & limits, + SizeLimits & leaf_limits, std::shared_ptr quota, const Names & column_names, const SelectQueryInfo & query_info, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index e780ebda111..4773652152e 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -583,6 +583,14 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( { std::atomic total_rows {0}; + SizeLimits limits; + /// bytes limit is ignored since we can't check it on this stage + limits = SizeLimits(settings.max_rows_to_read, 0, settings.read_overflow_mode); + + SizeLimits leaf_limits; + /// bytes limit is ignored since we can't check it on this stage + leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, 0, settings.read_overflow_mode_leaf); + auto process_part = [&](size_t part_index) { auto & part = parts[part_index]; @@ -610,18 +618,14 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (!ranges.ranges.empty()) { - if (settings.read_overflow_mode == OverflowMode::THROW && settings.max_rows_to_read) + if (settings.read_overflow_mode == OverflowMode::THROW && (limits.max_rows || leaf_limits.max_rows)) { /// Fail fast if estimated number of rows to read exceeds the limit auto current_rows_estimate = ranges.getRowsCount(); size_t prev_total_rows_estimate = total_rows.fetch_add(current_rows_estimate); size_t total_rows_estimate = current_rows_estimate + prev_total_rows_estimate; - if (total_rows_estimate > settings.max_rows_to_read) - throw Exception( - "Limit for rows (controlled by 'max_rows_to_read' setting) exceeded, max rows: " - + formatReadableQuantity(settings.max_rows_to_read) - + ", estimated rows to read (at least): " + formatReadableQuantity(total_rows_estimate), - ErrorCodes::TOO_MANY_ROWS); + limits.check(total_rows_estimate, 0, "rows (controlled by 'max_rows_to_read' setting)", ErrorCodes::TOO_MANY_ROWS); + leaf_limits.check(total_rows_estimate, 0, "rows (controlled by 'max_rows_to_read_leaf' setting)", ErrorCodes::TOO_MANY_ROWS); } parts_with_ranges[part_index] = std::move(ranges); diff --git a/tests/queries/0_stateless/01455_shard_leaf_max_rows_bytes_to_read.reference b/tests/queries/0_stateless/01455_shard_leaf_max_rows_bytes_to_read.reference new file mode 100644 index 00000000000..cccfb12c957 --- /dev/null +++ b/tests/queries/0_stateless/01455_shard_leaf_max_rows_bytes_to_read.reference @@ -0,0 +1,6 @@ +100 +100 +100 +100 +100000 +100000 diff --git a/tests/queries/0_stateless/01455_shard_leaf_max_rows_bytes_to_read.sql b/tests/queries/0_stateless/01455_shard_leaf_max_rows_bytes_to_read.sql new file mode 100755 index 00000000000..fca5c4534f7 --- /dev/null +++ b/tests/queries/0_stateless/01455_shard_leaf_max_rows_bytes_to_read.sql @@ -0,0 +1,29 @@ +SELECT count() FROM (SELECT * FROM remote('127.0.0.1', system.numbers) LIMIT 100) SETTINGS max_rows_to_read_leaf=1; -- { serverError 158 } +SELECT count() FROM (SELECT * FROM remote('127.0.0.1', system.numbers) LIMIT 100) SETTINGS max_bytes_to_read_leaf=1; -- { serverError 307 } +SELECT count() FROM (SELECT * FROM remote('127.0.0.1', system.numbers) LIMIT 100) SETTINGS max_rows_to_read_leaf=100; +SELECT count() FROM (SELECT * FROM remote('127.0.0.1', system.numbers) LIMIT 100) SETTINGS max_bytes_to_read_leaf=1000; + +SELECT count() FROM (SELECT * FROM remote('127.0.0.2', system.numbers) LIMIT 100) SETTINGS max_rows_to_read_leaf=1; -- { serverError 158 } +SELECT count() FROM (SELECT * FROM remote('127.0.0.2', system.numbers) LIMIT 100) SETTINGS max_bytes_to_read_leaf=1; -- { serverError 307 } +SELECT count() FROM (SELECT * FROM remote('127.0.0.2', system.numbers) LIMIT 100) SETTINGS max_rows_to_read_leaf=100; +SELECT count() FROM (SELECT * FROM remote('127.0.0.2', system.numbers) LIMIT 100) SETTINGS max_bytes_to_read_leaf=1000; + +DROP TABLE IF EXISTS test_local; +DROP TABLE IF EXISTS test_distributed; + +CREATE TABLE test_local (date Date, value UInt32) ENGINE = MergeTree(date, date, 8192); +CREATE TABLE test_distributed AS test_local ENGINE = Distributed(test_cluster_two_shards, currentDatabase(), test_local, rand()); + +INSERT INTO test_local SELECT '2000-08-01', number as value from numbers(50000); + +SELECT count() FROM (SELECT * FROM test_distributed) SETTINGS max_rows_to_read_leaf = 40000; -- { serverError 158 } +SELECT count() FROM (SELECT * FROM test_distributed) SETTINGS max_bytes_to_read_leaf = 40000; -- { serverError 307 } + +SELECT count() FROM (SELECT * FROM test_distributed) SETTINGS max_rows_to_read = 60000; -- { serverError 158 } +SELECT count() FROM (SELECT * FROM test_distributed) SETTINGS max_rows_to_read_leaf = 60000; + +SELECT count() FROM (SELECT * FROM test_distributed) SETTINGS max_bytes_to_read = 100000; -- { serverError 307 } +SELECT count() FROM (SELECT * FROM test_distributed) SETTINGS max_bytes_to_read_leaf = 100000; + +DROP TABLE IF EXISTS test_local; +DROP TABLE IF EXISTS test_distributed; \ No newline at end of file From 21bd45fb22985092d81a78a14de1ab321a055a0f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 17 Sep 2020 12:57:48 +0300 Subject: [PATCH 33/68] Update InterpreterSelectQuery.cpp --- src/Interpreters/InterpreterSelectQuery.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 3cbbdb576b3..cd2c16813b4 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1446,7 +1446,8 @@ void InterpreterSelectQuery::executeFetchColumns( /// Set the limits and quota for reading data, the speed and time of the query. - if (!options.ignore_limits) { + if (!options.ignore_limits) + { limits = getLimitsForStorage(settings, options); leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, settings.max_bytes_to_read_leaf, settings.read_overflow_mode_leaf); From ada1c7c4a03600de35f6031840dae25c9622025c Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 17 Sep 2020 13:15:46 +0300 Subject: [PATCH 34/68] fixup --- docker/test/performance-comparison/perf.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 64314c129b5..7966875f0f9 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -194,7 +194,7 @@ for conn_index, c in enumerate(connections): # Run the queries in randomized order, but preserve their indexes as specified # in the test XML. To avoid using too much time, limit the number of queries # we run per test. -queries_to_run = random.sample(range(0, len(test_queries)), args.max_queries or len(test_queries)) +queries_to_run = random.sample(range(0, len(test_queries)), min(len(test_queries), args.max_queries or len(test_queries))) # Run test queries. for query_index in queries_to_run: From 4773bf57eaab4b2890ff24e74616525d7b1ca571 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 17 Sep 2020 13:47:29 +0300 Subject: [PATCH 35/68] Use logical core id ('processor') for CPU freq metric See https://github.com/ClickHouse/ClickHouse/issues/14923 --- src/Interpreters/AsynchronousMetrics.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index ac71a88dc00..feb2036a0d6 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -332,7 +332,7 @@ void AsynchronousMetrics::update() ReadBufferFromFile buf("/proc/cpuinfo", 32768 /* buf_size */); // We need the following lines: - // core id : 4 + // processor : 4 // cpu MHz : 4052.941 // They contain tabs and are interspersed with other info. int core_id = 0; @@ -346,7 +346,7 @@ void AsynchronousMetrics::update() // It doesn't read the EOL itself. ++buf.position(); - if (s.rfind("core id", 0) == 0) + if (s.rfind("processor", 0) == 0) { if (auto colon = s.find_first_of(':')) { From a5560cd62dd9b6c058c71f0c9d72e11300b58a2d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 14:27:17 +0300 Subject: [PATCH 36/68] Better drop replicated tables --- src/Storages/StorageReplicatedMergeTree.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 608d983a21e..d4bbb66df9e 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -697,7 +697,9 @@ void StorageReplicatedMergeTree::drop() if (has_metadata_in_zookeeper) { - auto zookeeper = tryGetZooKeeper(); + /// Table can be shut down, restarting thread is not active + /// and calling StorageReplicatedMergeTree::getZooKeeper() won't suffice. + auto zookeeper = global_context.getZooKeeper(); /// If probably there is metadata in ZooKeeper, we don't allow to drop the table. if (is_readonly || !zookeeper) From 6c369c8dc669abaa590aaf275fe5e948a7aaa4f0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 17 Sep 2020 15:01:03 +0300 Subject: [PATCH 37/68] Fix merge error --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp | 9 +++++---- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a3402dae735..808d8c514d1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1874,7 +1874,7 @@ bool MergeTreeData::renameTempPartAndReplace( part_info.mutation = 0; /// it's equal to min_block by default part_name = part->getNewName(part_info); } - else + else /// Parts from ReplicatedMergeTree already have names part_name = part->name; LOG_TRACE(log, "Renaming temporary part {} to {}.", part->relative_path, part_name); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index eac5f4c74ae..d1c3a4f3883 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -120,7 +120,6 @@ void ReplicatedMergeTreeBlockOutputStream::write(const Block & block) { last_block_is_duplicate = false; - /// TODO Is it possible to not lock the table structure here? storage.delayInsertOrThrowIfNeeded(&storage.partial_shutdown_event); auto zookeeper = storage.getZooKeeper(); @@ -423,16 +422,18 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( part->state = MergeTreeDataPartState::Temporary; part->renameTo(temporary_part_relative_path, false); + /// If this part appeared on other replica than it's better to try to write it locally one more time. If it's our part + /// than it will be ignored on the next itration. ++loop_counter; if (loop_counter == max_iterations) - throw Exception("Too many transaction retires - it may indicate an error", ErrorCodes::DUPLICATE_DATA_PART); + throw Exception("Too many transaction retries - it may indicate an error", ErrorCodes::DUPLICATE_DATA_PART); continue; } else if (multi_code == Coordination::Error::ZNODEEXISTS && failed_op_path == block_id_path) { /// Block with the same id have just appeared in table (or other replica), rollback the insertion. - LOG_INFO(log, "Block with ID {} already exists; ignoring it (removing part {})", block_id, part->name); - throw Exception("Another quorum insert has been already started", ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE); + transaction.rollback(); + throw Exception("Another quorum insert has been already started", ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE); } else { From 6601f4f7866749f7dc38e755c026b6c7b5d0550b Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 17 Sep 2020 15:10:06 +0300 Subject: [PATCH 38/68] Better errors, fix merge --- .../MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index d1c3a4f3883..9c44db614d8 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -370,14 +370,13 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( { if (is_already_existing_part) { - LOG_INFO(log, "Part {} is duplicate and it is already written by concurrent request or fetched; ignoring it.", - block_id, existing_part_name); + LOG_INFO(log, "Part {} is duplicate and it is already written by concurrent request or fetched; ignoring it.", part->name); return; } else - throw Exception("Part with name {} is already written by concurrent request." + throw Exception(ErrorCodes::LOGICAL_ERROR, "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); + part->name); } Coordination::Responses responses; @@ -429,7 +428,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( throw Exception("Too many transaction retries - it may indicate an error", ErrorCodes::DUPLICATE_DATA_PART); continue; } - else if (multi_code == Coordination::Error::ZNODEEXISTS && failed_op_path == block_id_path) + else if (multi_code == Coordination::Error::ZNODEEXISTS && failed_op_path == quorum_info.status_path) { /// Block with the same id have just appeared in table (or other replica), rollback the insertion. transaction.rollback(); From 22bd22702e7b68e0182c81abc6e4a2820f8a1132 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 17 Sep 2020 15:59:14 +0200 Subject: [PATCH 39/68] Fix enable_optimize_predicate_expression for finalizeAggregation finalizeAggregation was wrongly marked as stateful, preventing pushing the conditions down. --- src/Functions/finalizeAggregation.cpp | 5 --- .../00808_not_optimize_predicate.reference | 17 +++++++++ .../00808_not_optimize_predicate.sql | 36 +++++++++++++++++++ 3 files changed, 53 insertions(+), 5 deletions(-) diff --git a/src/Functions/finalizeAggregation.cpp b/src/Functions/finalizeAggregation.cpp index 51afb4729dc..ae2a67dec20 100644 --- a/src/Functions/finalizeAggregation.cpp +++ b/src/Functions/finalizeAggregation.cpp @@ -34,11 +34,6 @@ public: return name; } - bool isStateful() const override - { - return true; - } - size_t getNumberOfArguments() const override { return 1; diff --git a/tests/queries/0_stateless/00808_not_optimize_predicate.reference b/tests/queries/0_stateless/00808_not_optimize_predicate.reference index 1899fa2045d..d8ab9425aab 100644 --- a/tests/queries/0_stateless/00808_not_optimize_predicate.reference +++ b/tests/queries/0_stateless/00808_not_optimize_predicate.reference @@ -10,3 +10,20 @@ (1,1) (3,2) (5,2) +-------finalizeAggregation should not be stateful (issue #14847)------- +2 62 +3 87 +4 112 +5 137 +SELECT + n, + `finalizeAggregation(s)` +FROM +( + SELECT + n, + finalizeAggregation(s) + FROM test_00808_push_down_with_finalizeAggregation + WHERE (n <= 5) AND (n >= 2) +) +WHERE (n >= 2) AND (n <= 5) diff --git a/tests/queries/0_stateless/00808_not_optimize_predicate.sql b/tests/queries/0_stateless/00808_not_optimize_predicate.sql index ad3df16e4bb..579fd00cb8c 100644 --- a/tests/queries/0_stateless/00808_not_optimize_predicate.sql +++ b/tests/queries/0_stateless/00808_not_optimize_predicate.sql @@ -36,3 +36,39 @@ SELECT arrayJoin(arrayMap(x -> x, arraySort(groupArray((ts, n))))) AS k FROM ( DROP TABLE IF EXISTS test_00808; + +SELECT '-------finalizeAggregation should not be stateful (issue #14847)-------'; + +DROP TABLE IF EXISTS test_00808_push_down_with_finalizeAggregation; + +CREATE TABLE test_00808_push_down_with_finalizeAggregation ENGINE = AggregatingMergeTree +ORDER BY n AS +SELECT + intDiv(number, 25) AS n, + avgState(number) AS s +FROM numbers(2500) +GROUP BY n; + +SET force_primary_key = 1, enable_debug_queries = 1, enable_optimize_predicate_expression = 1; + +SELECT * +FROM +( + SELECT + n, + finalizeAggregation(s) + FROM test_00808_push_down_with_finalizeAggregation +) +WHERE (n >= 2) AND (n <= 5); + +ANALYZE SELECT * +FROM +( + SELECT + n, + finalizeAggregation(s) + FROM test_00808_push_down_with_finalizeAggregation +) +WHERE (n >= 2) AND (n <= 5); + +DROP TABLE IF EXISTS test_00808_push_down_with_finalizeAggregation; From d09fd8ca8b90cd3f6c128964dcc2cc29cc07bb26 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 17 Sep 2020 17:01:17 +0300 Subject: [PATCH 40/68] Add comment --- .../MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 9c44db614d8..694d2b4b411 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -416,7 +416,6 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( transaction.rollback(); - part->is_duplicate = true; part->is_temp = true; part->state = MergeTreeDataPartState::Temporary; part->renameTo(temporary_part_relative_path, false); @@ -425,12 +424,14 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( /// than it will be ignored on the next itration. ++loop_counter; if (loop_counter == max_iterations) + { + part->is_duplicate = true; /// Part is duplicate, just remove it from local FS throw Exception("Too many transaction retries - it may indicate an error", ErrorCodes::DUPLICATE_DATA_PART); + } continue; } else if (multi_code == Coordination::Error::ZNODEEXISTS && failed_op_path == quorum_info.status_path) { - /// Block with the same id have just appeared in table (or other replica), rollback the insertion. transaction.rollback(); throw Exception("Another quorum insert has been already started", ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE); } @@ -463,6 +464,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( { if (is_already_existing_part) { + /// We get duplicate part without fetch storage.updateQuorum(part->name); } From 3da806f8f6c078113038541a72c30769940b8282 Mon Sep 17 00:00:00 2001 From: Vxider Date: Thu, 17 Sep 2020 22:49:27 +0800 Subject: [PATCH 41/68] Throws LOGICAL_ERROR if argument is not a literal --- src/TableFunctions/TableFunctionNull.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionNull.cpp b/src/TableFunctions/TableFunctionNull.cpp index fe9c2d36d92..d3fccb3d385 100644 --- a/src/TableFunctions/TableFunctionNull.cpp +++ b/src/TableFunctions/TableFunctionNull.cpp @@ -14,6 +14,7 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; } StoragePtr TableFunctionNull::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const @@ -25,7 +26,10 @@ StoragePtr TableFunctionNull::executeImpl(const ASTPtr & ast_function, const Con if (arguments.size() != 1) throw Exception("Table function '" + getName() + "' requires 'structure'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - auto structure = arguments[0]->as().value.safeGet(); + const auto * literal = arguments[0]->as(); + if (!literal) + throw Exception("Table function " + getName() + " requested literal argument.", ErrorCodes::LOGICAL_ERROR); + auto structure = literal->value.safeGet(); ColumnsDescription columns = parseColumnsListFromString(structure, context); auto res = StorageNull::create(StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription()); From c156c0f28fee321c3b4c6d686b5fe56243f64adb Mon Sep 17 00:00:00 2001 From: yulu86 Date: Thu, 17 Sep 2020 22:54:28 +0800 Subject: [PATCH 42/68] modify syntax --- docs/zh/sql-reference/functions/conditional-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/sql-reference/functions/conditional-functions.md b/docs/zh/sql-reference/functions/conditional-functions.md index eabe253ab1c..265c4387cb1 100644 --- a/docs/zh/sql-reference/functions/conditional-functions.md +++ b/docs/zh/sql-reference/functions/conditional-functions.md @@ -34,7 +34,7 @@ │ 2 │ 3 │ └───┴──────┘ -执行查询 `SELECT multiIf(isNull(y) x, y < 3, y, NULL) FROM t_null`。结果: +执行查询 `SELECT multiIf(isNull(y), x, y < 3, y, NULL) FROM t_null`。结果: ┌─multiIf(isNull(y), x, less(y, 3), y, NULL)─┐ │ 1 │ From eab11e35cd93d992283fbff3e401c76f24b31ece Mon Sep 17 00:00:00 2001 From: yulu86 Date: Thu, 17 Sep 2020 22:58:20 +0800 Subject: [PATCH 43/68] modify title of aggragate functions in Chinese --- docs/zh/sql-reference/aggregate-functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/sql-reference/aggregate-functions/index.md b/docs/zh/sql-reference/aggregate-functions/index.md index 57d8e362d99..436a8f433ea 100644 --- a/docs/zh/sql-reference/aggregate-functions/index.md +++ b/docs/zh/sql-reference/aggregate-functions/index.md @@ -1,6 +1,6 @@ --- toc_priority: 33 -toc_title: 简介 +toc_title: 聚合函数 --- # 聚合函数 {#aggregate-functions} From b07c00ef7ccdd64b33fccf548b8780691baf5597 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 17 Sep 2020 18:33:50 +0300 Subject: [PATCH 44/68] Ugly, but probably working implementation --- src/Storages/MergeTree/MergeTreeData.cpp | 35 +++++++++++++++++++ src/Storages/MergeTree/MergeTreeData.h | 5 +++ .../ReplicatedMergeTreeBlockOutputStream.cpp | 5 ++- 3 files changed, 42 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 808d8c514d1..1ce611faf61 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1981,6 +1981,22 @@ void MergeTreeData::removePartsFromWorkingSet(const MergeTreeData::DataPartsVect } } +void MergeTreeData::removePartsFromWorkingSetImmediatelyAndSetTemporaryState(const DataPartsVector & remove) +{ + auto lock = lockParts(); + + for (const auto & part : remove) + { + auto it_part = data_parts_by_info.find(part->info); + if (it_part == data_parts_by_info.end()) + throw Exception("Part " + part->getNameWithState() + " not found in data_parts", ErrorCodes::LOGICAL_ERROR); + + modifyPartState(part, IMergeTreeDataPart::State::Temporary); + /// Erase immediately + data_parts_indexes.erase(it_part); + } +} + void MergeTreeData::removePartsFromWorkingSet(const DataPartsVector & remove, bool clear_without_timeout, DataPartsLock * acquired_lock) { auto lock = (acquired_lock) ? DataPartsLock() : lockParts(); @@ -3100,6 +3116,25 @@ MergeTreeData::DataPartPtr MergeTreeData::getAnyPartInPartition( return nullptr; } + +void MergeTreeData::Transaction::rollbackPartsToTemporaryState() +{ + if (!isEmpty()) + { + std::stringstream ss; + ss << " Rollbacking parts state to temporary and removing from working set:"; + for (const auto & part : precommitted_parts) + ss << " " << part->relative_path; + ss << "."; + LOG_DEBUG(data.log, "Undoing transaction.{}", ss.str()); + + data.removePartsFromWorkingSetImmediatelyAndSetTemporaryState( + DataPartsVector(precommitted_parts.begin(), precommitted_parts.end())); + } + + clear(); +} + void MergeTreeData::Transaction::rollback() { if (!isEmpty()) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 628b394ee05..33e79af0952 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -224,6 +224,8 @@ public: DataPartsVector commit(MergeTreeData::DataPartsLock * acquired_parts_lock = nullptr); void rollback(); + void rollbackPartsToTemporaryState(); + size_t size() const { return precommitted_parts.size(); } bool isEmpty() const { return precommitted_parts.empty(); } @@ -440,6 +442,9 @@ public: MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction, DataPartsLock & lock, DataPartsVector * out_covered_parts = nullptr); + + void removePartsFromWorkingSetImmediatelyAndSetTemporaryState(const DataPartsVector & remove); + /// Removes parts from the working set parts. /// Parts in add must already be in data_parts with PreCommitted, Committed, or Outdated states. /// If clear_without_timeout is true, the parts will be deleted at once, or during the next call to diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 694d2b4b411..dbc77ba74c1 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -343,10 +343,10 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( part->name = existing_part_name; part->info = MergeTreePartInfo::fromPartName(existing_part_name, storage.format_version); - /// Used only for exception messages. block_number = part->info.min_block; + /// Do not check for duplicate on commit to ZK. block_id_path.clear(); } @@ -414,10 +414,9 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( 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_relative_path); - transaction.rollback(); + transaction.rollbackPartsToTemporaryState(); part->is_temp = true; - part->state = MergeTreeDataPartState::Temporary; part->renameTo(temporary_part_relative_path, false); /// If this part appeared on other replica than it's better to try to write it locally one more time. If it's our part From 4fc86f05dad66af5a3b01382459d2b4a21931ab6 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 17 Sep 2020 19:21:59 +0300 Subject: [PATCH 45/68] some fixes + docs + report queries that are not short --- docker/test/performance-comparison/README.md | 56 +++++++++++++++---- docker/test/performance-comparison/compare.sh | 38 ++++++++----- docker/test/performance-comparison/perf.py | 19 ++++--- docker/test/performance-comparison/report.py | 39 +++++++------ 4 files changed, 104 insertions(+), 48 deletions(-) diff --git a/docker/test/performance-comparison/README.md b/docker/test/performance-comparison/README.md index d91cd9421ea..ec5691488f0 100644 --- a/docker/test/performance-comparison/README.md +++ b/docker/test/performance-comparison/README.md @@ -30,25 +30,47 @@ The report page itself constists of a several tables. Some of them always signif #### Tested Commits Informational, no action required. Log messages for the commits that are tested. Note that for the right commit, we show nominal tested commit `pull/*/head` and real tested commit `pull/*/merge`, which is generated by GitHub by merging latest master to the `pull/*/head` and which we actually build and test in CI. +#### Error Summary +Action required for every item. + +This table summarizes all errors that ocurred during the test. Click the links to go to the description of a particular error. + #### Run Errors -Action required for every item -- these are errors that must be fixed. The errors that ocurred when running some test queries. For more information about the error, download test output archive and see `test-name-err.log`. To reproduce, see 'How to run' below. +Action required for every item -- these are errors that must be fixed. + +The errors that ocurred when running some test queries. For more information about the error, download test output archive and see `test-name-err.log`. To reproduce, see 'How to run' below. #### Slow on Client -Action required for every item -- these are errors that must be fixed. This table shows queries that take significantly longer to process on the client than on the server. A possible reason might be sending too much data to the client, e.g., a forgotten `format Null`. +Action required for every item -- these are errors that must be fixed. + +This table shows queries that take significantly longer to process on the client than on the server. A possible reason might be sending too much data to the client, e.g., a forgotten `format Null`. + +#### Inconsistent Short Marking +Action required for every item -- these are errors that must be fixed. + +Queries that have "short" duration (on the order of 0.1 s) can't be reliably tested in a normal way, where we perform a small (about ten) measurements for each server, because the signal-to-noise ratio is much smaller. There is a special mode for such queries that instead runs them for a fixed amount of time, normally with much higher number of measurements (up to thousands). This mode must be explicitly enabled by the test author to avoid accidental errors. It must be used only for queries that are meant to complete "immediately", such as `select count(*)`. If your query is not supposed to be "immediate", try to make it run longer, by e.g. processing more data. + +This table shows queries for which the "short" marking is not consistent with the actual query run time -- i.e., a query runs for a long time but is marked as short, or it runs very fast but is not marked as short. + +If your query is really supposed to complete "immediately" and can't be made to run longer, you have to mark it as "short". To do so, write `...` in the test file. The value of "short" attribute is evaluated as a python expression, and substitutions are performed, so you can write something like `select count(*) from table where {column1} > {column2}`, to mark only a particular combination of variables as short. -#### Short Queries not Marked as Short -Action required for every item -- these are errors that must be fixed. This table shows queries that are "short" but not explicitly marked as such. "Short" queries are too fast to meaningfully compare performance, because the changes are drowned by the noise. We consider all queries that run faster than 0.02 s to be "short", and only check the performance if they became slower than this threshold. Probably this mode is not what you want, so you have to increase the query run time to be between 1 and 0.1 s, so that the performance can be compared. You do want this "short" mode for queries that complete "immediately", such as some varieties of `select count(*)`. You have to mark them as "short" explicitly by writing `...`. The value of "short" attribute is evaluated as a python expression, and substitutions are performed, so you can write something like `select count(*) from table where {column1} > {column2}`, to mark only a particular combination of variables as short. #### Partial Queries -Action required for the cells marked in red. Shows the queries we are unable to run on an old server -- probably because they contain a new function. You should see this table when you add a new function and a performance test for it. Check that the run time and variance are acceptable (run time between 0.1 and 1 seconds, variance below 10%). If not, they will be highlighted in red. +Action required for the cells marked in red. + +Shows the queries we are unable to run on an old server -- probably because they contain a new function. You should see this table when you add a new function and a performance test for it. Check that the run time and variance are acceptable (run time between 0.1 and 1 seconds, variance below 10%). If not, they will be highlighted in red. #### Changes in Performance -Action required for the cells marked in red, and some cheering is appropriate for the cells marked in green. These are the queries for which we observe a statistically significant change in performance. Note that there will always be some false positives -- we try to filter by p < 0.001, and have 2000 queries, so two false positives per run are expected. In practice we have more -- e.g. code layout changed because of some unknowable jitter in compiler internals, so the change we observe is real, but it is a 'false positive' in the sense that it is not directly caused by your changes. If, based on your knowledge of ClickHouse internals, you can decide that the observed test changes are not relevant to the changes made in the tested PR, you can ignore them. +Action required for the cells marked in red, and some cheering is appropriate for the cells marked in green. + +These are the queries for which we observe a statistically significant change in performance. Note that there will always be some false positives -- we try to filter by p < 0.001, and have 2000 queries, so two false positives per run are expected. In practice we have more -- e.g. code layout changed because of some unknowable jitter in compiler internals, so the change we observe is real, but it is a 'false positive' in the sense that it is not directly caused by your changes. If, based on your knowledge of ClickHouse internals, you can decide that the observed test changes are not relevant to the changes made in the tested PR, you can ignore them. You can find flame graphs for queries with performance changes in the test output archive, in files named as 'my_test_0_Cpu_SELECT 1 FROM....FORMAT Null.left.svg'. First goes the test name, then the query number in the test, then the trace type (same as in `system.trace_log`), and then the server version (left is old and right is new). #### Unstable Queries -Action required for the cells marked in red. These are queries for which we did not observe a statistically significant change in performance, but for which the variance in query performance is very high. This means that we are likely to observe big changes in performance even in the absence of real changes, e.g. when comparing the server to itself. Such queries are going to have bad sensitivity as performance tests -- if a query has, say, 50% expected variability, this means we are going to see changes in performance up to 50%, even when there were no real changes in the code. And because of this, we won't be able to detect changes less than 50% with such a query, which is pretty bad. The reasons for the high variability must be investigated and fixed; ideally, the variability should be brought under 5-10%. +Action required for the cells marked in red. + +These are the queries for which we did not observe a statistically significant change in performance, but for which the variance in query performance is very high. This means that we are likely to observe big changes in performance even in the absence of real changes, e.g. when comparing the server to itself. Such queries are going to have bad sensitivity as performance tests -- if a query has, say, 50% expected variability, this means we are going to see changes in performance up to 50%, even when there were no real changes in the code. And because of this, we won't be able to detect changes less than 50% with such a query, which is pretty bad. The reasons for the high variability must be investigated and fixed; ideally, the variability should be brought under 5-10%. The most frequent reason for instability is that the query is just too short -- e.g. below 0.1 seconds. Bringing query time to 0.2 seconds or above usually helps. Other reasons may include: @@ -58,18 +80,30 @@ Other reasons may include: Investigating the instablility is the hardest problem in performance testing, and we still have not been able to understand the reasons behind the instability of some queries. There are some data that can help you in the performance test output archive. Look for files named 'my_unstable_test_0_SELECT 1...FORMAT Null.{left,right}.metrics.rep'. They contain metrics from `system.query_log.ProfileEvents` and functions from stack traces from `system.trace_log`, that vary significantly between query runs. The second column is array of \[min, med, max] values for the metric. Say, if you see `PerfCacheMisses` there, it may mean that the code being tested has not-so-cache-local memory access pattern that is sensitive to memory layout. #### Skipped Tests -Informational, no action required. Shows the tests that were skipped, and the reason for it. Normally it is because the data set required for the test was not loaded, or the test is marked as 'long' -- both cases mean that the test is too big to be ran per-commit. +Informational, no action required. + +Shows the tests that were skipped, and the reason for it. Normally it is because the data set required for the test was not loaded, or the test is marked as 'long' -- both cases mean that the test is too big to be ran per-commit. #### Test Performance Changes -Informational, no action required. This table summarizes the changes in performance of queries in each test -- how many queries have changed, how many are unstable, and what is the magnitude of the changes. +Informational, no action required. + +This table summarizes the changes in performance of queries in each test -- how many queries have changed, how many are unstable, and what is the magnitude of the changes. #### Test Times -Action required for the cells marked in red. This table shows the run times for all the tests. You may have to fix two kinds of errors in this table: +Action required for the cells marked in red. + +This table shows the run times for all the tests. You may have to fix two kinds of errors in this table: 1) Average query run time is too long -- probalby means that the preparatory steps such as creating the table and filling them with data are taking too long. Try to make them faster. 2) Longest query run time is too long -- some particular queries are taking too long, try to make them faster. The ideal query run time is between 0.1 and 1 s. #### Metric Changes -No action required. These are changes in median values of metrics from `system.asynchronous_metrics_log`. These metrics are prone to unexplained variation and you can safely ignore this table unless it's interesting to you for some particular reason (e.g. you want to compare memory usage). There are also graphs of these metrics in the performance test output archive, in the `metrics` folder. +No action required. + +These are changes in median values of metrics from `system.asynchronous_metrics_log`. These metrics are prone to unexplained variation and you can safely ignore this table unless it's interesting to you for some particular reason (e.g. you want to compare memory usage). There are also graphs of these metrics in the performance test output archive, in the `metrics` folder. + +#### Errors while Building the Report +Ask a maintainer for help. These errors normally indicate a problem with testing infrastructure. + ### How to Run Run the entire docker container, specifying PR number (0 for master) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index a65b13775a9..46a7215e0e6 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -466,8 +466,6 @@ build_log_column_definitions cat analyze/errors.log >> report/errors.log ||: cat profile-errors.log >> report/errors.log ||: -short_query_threshold="0.02" - clickhouse-local --query " create view query_display_names as select * from file('analyze/query-display-names.tsv', TSV, @@ -653,11 +651,14 @@ create table queries_for_flamegraph engine File(TSVWithNamesAndTypes, select test, query_index from queries where unstable_show or changed_show ; --- List of queries that have 'short' duration, but are not marked as 'short' by --- the test author (we report them). -create table unmarked_short_queries_report - engine File(TSV, 'report/unmarked-short-queries.tsv') - as select time, test, query_index, query_display_name + +create view shortness + as select + (test, query_index) in + (select * from file('analyze/marked-short-queries.tsv', TSV, + 'test text, query_index int')) + as marked_short, + time, test, query_index, query_display_name from ( select right time, test, query_index from queries union all @@ -666,14 +667,25 @@ create table unmarked_short_queries_report left join query_display_names on times.test = query_display_names.test and times.query_index = query_display_names.query_index - where - (test, query_index) not in - (select * from file('analyze/marked-short-queries.tsv', TSV, - 'test text, query_index int')) - and time < $short_query_threshold - order by test, query_index ; +-- Report of queries that have inconsistent 'short' markings: +-- 1) have short duration, but are not marked as 'short' +-- 2) the reverse -- marked 'short' but take too long. +-- The threshold for 2) is twice the threshold for 1), to avoid jitter. +create table inconsistent_short_marking_report + engine File(TSV, 'report/inconsistent-short-marking.tsv') + as select + multiIf(marked_short and time > 0.1, 'marked as short but is too long', + not marked_short and time < 0.02, 'is short but not marked as such', + '') problem, + marked_short, time, + test, query_index, query_display_name + from shortness + where problem != '' + ; + + -------------------------------------------------------------------------------- -- various compatibility data formats follow, not related to the main report diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index d54d6444ee0..8021c4ac880 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -266,10 +266,22 @@ for query_index, q in enumerate(test_queries): server_seconds += c.last_query.elapsed print(f'query\t{query_index}\t{run_id}\t{conn_index}\t{c.last_query.elapsed}') + if c.last_query.elapsed > 10: + # Stop processing pathologically slow queries, to avoid timing out + # the entire test task. This shouldn't really happen, so we don't + # need much handling for this case and can just exit. + print(f'The query no. {query_index} is taking too long to run ({c.last_query.elapsed} s)', file=sys.stderr) + exit(2) + # Be careful with the counter, after this line it's the next iteration # already. run += 1 + # Try to run any query for at least the specified number of times, + # before considering other stop conditions. + if run < arg.runs: + continue + # For very short queries we have a special mode where we run them for at # least some time. The recommended lower bound of run time for "normal" # queries is about 0.1 s, and we run them about 10 times, giving the @@ -286,13 +298,6 @@ for query_index, q in enumerate(test_queries): if run >= args.runs: break - if c.last_query.elapsed > 10: - # Stop processing pathologically slow queries, to avoid timing out - # the entire test task. This shouldn't really happen, so we don't - # need much handling for this case and can just exit. - print(f'The query no. {query_index} is taking too long to run ({c.last_query.elapsed} s)', file=sys.stderr) - exit(2) - client_seconds = time.perf_counter() - start_seconds print(f'client-time\t{query_index}\t{client_seconds}\t{server_seconds}') diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index df346d1bdaa..e8481f77340 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -98,6 +98,9 @@ th {{ tr:nth-child(odd) td {{filter: brightness(90%);}} +.inconsistent-short-marking tr :nth-child(2), +.inconsistent-short-marking tr :nth-child(3), +.inconsistent-short-marking tr :nth-child(5), .all-query-times tr :nth-child(1), .all-query-times tr :nth-child(2), .all-query-times tr :nth-child(3), @@ -204,9 +207,11 @@ def tableStart(title): global table_anchor table_anchor = cls anchor = currentTableAnchor() + help_anchor = '-'.join(title.lower().split(' ')); return f"""

{title} + ?

""" @@ -249,7 +254,7 @@ def addSimpleTable(caption, columns, rows, pos=None): def add_tested_commits(): global report_errors try: - addSimpleTable('Tested commits', ['Old', 'New'], + addSimpleTable('Tested Commits', ['Old', 'New'], [['
{}
'.format(x) for x in [open('left-commit.txt').read(), open('right-commit.txt').read()]]]) @@ -275,7 +280,7 @@ def add_report_errors(): if not report_errors: return - text = tableStart('Errors while building the report') + text = tableStart('Errors while Building the Report') text += tableHeader(['Error']) for x in report_errors: text += tableRow([x]) @@ -289,7 +294,7 @@ def add_errors_explained(): return text = '' - text += tableStart('Error summary') + text += tableStart('Error Summary') text += tableHeader(['Description']) for row in errors_explained: text += tableRow(row) @@ -307,26 +312,26 @@ if args.report == 'main': run_error_rows = tsvRows('run-errors.tsv') error_tests += len(run_error_rows) - addSimpleTable('Run errors', ['Test', 'Error'], run_error_rows) + addSimpleTable('Run Errors', ['Test', 'Error'], run_error_rows) if run_error_rows: errors_explained.append([f'There were some errors while running the tests']); slow_on_client_rows = tsvRows('report/slow-on-client.tsv') error_tests += len(slow_on_client_rows) - addSimpleTable('Slow on client', + addSimpleTable('Slow on Client', ['Client time, s', 'Server time, s', 'Ratio', 'Test', 'Query'], slow_on_client_rows) if slow_on_client_rows: errors_explained.append([f'Some queries are taking noticeable time client-side (missing `FORMAT Null`?)']); - unmarked_short_rows = tsvRows('report/unmarked-short-queries.tsv') + unmarked_short_rows = tsvRows('report/inconsistent-short-marking.tsv') error_tests += len(unmarked_short_rows) - addSimpleTable('Short queries not marked as short', - ['New client time, s', 'Test', '#', 'Query'], + addSimpleTable('Inconsistent Short Marking', + ['Problem', 'Is marked as short', 'New client time, s', 'Test', '#', 'Query'], unmarked_short_rows) if unmarked_short_rows: - errors_explained.append([f'Some queries have short duration but are not explicitly marked as "short"']); + errors_explained.append([f'Some queries have inconsistent short marking']); def add_partial(): rows = tsvRows('report/partial-queries-report.tsv') @@ -334,7 +339,7 @@ if args.report == 'main': return global unstable_partial_queries, slow_average_tests, tables - text = tableStart('Partial queries') + text = tableStart('Partial Queries') columns = ['Median time, s', 'Relative time variance', 'Test', '#', 'Query'] text += tableHeader(columns) attrs = ['' for c in columns] @@ -365,7 +370,7 @@ if args.report == 'main': global faster_queries, slower_queries, tables - text = tableStart('Changes in performance') + text = tableStart('Changes in Performance') columns = [ 'Old, s', # 0 'New, s', # 1 @@ -422,7 +427,7 @@ if args.report == 'main': 'Query' #7 ] - text = tableStart('Unstable queries') + text = tableStart('Unstable Queries') text += tableHeader(columns) attrs = ['' for c in columns] @@ -443,9 +448,9 @@ if args.report == 'main': add_unstable_queries() skipped_tests_rows = tsvRows('analyze/skipped-tests.tsv') - addSimpleTable('Skipped tests', ['Test', 'Reason'], skipped_tests_rows) + addSimpleTable('Skipped Tests', ['Test', 'Reason'], skipped_tests_rows) - addSimpleTable('Test performance changes', + addSimpleTable('Test Performance Changes', ['Test', 'Ratio of speedup (-) or slowdown (+)', 'Queries', 'Total not OK', 'Changed perf', 'Unstable'], tsvRows('report/test-perf-changes.tsv')) @@ -465,7 +470,7 @@ if args.report == 'main': 'Shortest query
(sum for all runs), s', #6 ] - text = tableStart('Test times') + text = tableStart('Test Times') text += tableHeader(columns) nominal_runs = 7 # FIXME pass this as an argument @@ -496,7 +501,7 @@ if args.report == 'main': add_test_times() - addSimpleTable('Metric changes', + addSimpleTable('Metric Changes', ['Metric', 'Old median value', 'New median value', 'Relative difference', 'Times difference'], tsvRows('metrics/changes.tsv')) @@ -587,7 +592,7 @@ elif args.report == 'all-queries': 'Query', #9 ] - text = tableStart('All query times') + text = tableStart('All Query Times') text += tableHeader(columns) attrs = ['' for c in columns] From 0275202167945b28f90a5ec84c25fcf3596766ac Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 17 Sep 2020 21:31:39 +0300 Subject: [PATCH 46/68] Update jemalloc to include patch for percpu arena Refs: https://github.com/jemalloc/jemalloc/pull/1676 --- .gitmodules | 2 +- contrib/jemalloc | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index 19f93ee8270..eb21c4bfd00 100644 --- a/.gitmodules +++ b/.gitmodules @@ -37,7 +37,7 @@ url = https://github.com/ClickHouse-Extras/mariadb-connector-c.git [submodule "contrib/jemalloc"] path = contrib/jemalloc - url = https://github.com/jemalloc/jemalloc.git + url = https://github.com/ClickHouse-Extras/jemalloc.git [submodule "contrib/unixodbc"] path = contrib/unixodbc url = https://github.com/ClickHouse-Extras/UnixODBC.git diff --git a/contrib/jemalloc b/contrib/jemalloc index ea6b3e973b4..026764f1999 160000 --- a/contrib/jemalloc +++ b/contrib/jemalloc @@ -1 +1 @@ -Subproject commit ea6b3e973b477b8061e0076bb257dbd7f3faa756 +Subproject commit 026764f19995c53583ab25a3b9c06a2fd74e4689 From dc677b93fb2e803c6a369c66e960cf9308446f04 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 17 Sep 2020 22:30:17 +0300 Subject: [PATCH 47/68] Comments and fix test --- src/Storages/MergeTree/MergeTreeData.h | 7 ++++++- .../queries/0_stateless/01459_manual_write_to_replicas.sh | 2 +- .../0_stateless/01459_manual_write_to_replicas_quorum.sh | 2 +- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 33e79af0952..1179d029ff5 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -224,8 +224,10 @@ public: DataPartsVector commit(MergeTreeData::DataPartsLock * acquired_parts_lock = nullptr); void rollback(); - void rollbackPartsToTemporaryState(); + /// Immediately remove parts from table's data_parts set and change part + /// state to temporary. Useful for new parts which not present in table. + void rollbackPartsToTemporaryState(); size_t size() const { return precommitted_parts.size(); } bool isEmpty() const { return precommitted_parts.empty(); } @@ -443,6 +445,9 @@ public: DataPartsVector * out_covered_parts = nullptr); + /// Remove parts from working set immediately (without wait for background + /// process). Transfer part state to temporary. Have very limited usage only + /// for new parts which don't already present in table. void removePartsFromWorkingSetImmediatelyAndSetTemporaryState(const DataPartsVector & remove); /// Removes parts from the working set parts. diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh index 42f5dc1bbf3..c402e19c3dc 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh @@ -10,7 +10,7 @@ NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do $CLICKHOUSE_CLIENT -n -q " DROP TABLE IF EXISTS r$i; - CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/r', 'r$i') ORDER BY x; + CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/01459_manual_write_ro_replicas/r', 'r$i') ORDER BY x; " done diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh index e6709b76316..48cf034a4c3 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh @@ -10,7 +10,7 @@ NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do $CLICKHOUSE_CLIENT -n -q " DROP TABLE IF EXISTS r$i; - CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/r', 'r$i') ORDER BY x; + CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/01459_manual_write_ro_replicas_quorum/r', 'r$i') ORDER BY x; " done From 447205c5258ccc65f3305461541a2e60e179115e Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 17 Sep 2020 22:30:45 +0300 Subject: [PATCH 48/68] More comments --- src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index dbc77ba74c1..5696a9cf890 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -414,6 +414,8 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( 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_relative_path); + /// We will try to add this part again on the new iteration as it's just a new part. + /// So remove it from storage parts set immediately and transfer state to temporary. transaction.rollbackPartsToTemporaryState(); part->is_temp = true; From ddca262fe63445d9ea077b773926692b25356880 Mon Sep 17 00:00:00 2001 From: roman Date: Thu, 17 Sep 2020 20:54:21 +0100 Subject: [PATCH 49/68] fix review comments --- src/Processors/Sources/SourceWithProgress.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 10 +++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Processors/Sources/SourceWithProgress.cpp b/src/Processors/Sources/SourceWithProgress.cpp index e3dd614eece..021bf04d86d 100644 --- a/src/Processors/Sources/SourceWithProgress.cpp +++ b/src/Processors/Sources/SourceWithProgress.cpp @@ -93,8 +93,8 @@ void SourceWithProgress::progress(const Progress & value) } } - if (!leaf_limits.check(rows_to_check_limit, progress.read_bytes,"rows or bytes to read on leaf node", - ErrorCodes::TOO_MANY_ROWS, ErrorCodes::TOO_MANY_BYTES)) + if (!leaf_limits.check(rows_to_check_limit, progress.read_bytes, "rows or bytes to read on leaf node", + ErrorCodes::TOO_MANY_ROWS, ErrorCodes::TOO_MANY_BYTES)) { cancel(); } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 4773652152e..7bfd6574b8a 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -584,12 +584,12 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( std::atomic total_rows {0}; SizeLimits limits; - /// bytes limit is ignored since we can't check it on this stage - limits = SizeLimits(settings.max_rows_to_read, 0, settings.read_overflow_mode); + if (settings.read_overflow_mode == OverflowMode::THROW && settings.max_rows_to_read) + limits = SizeLimits(settings.max_rows_to_read, 0, settings.read_overflow_mode); SizeLimits leaf_limits; - /// bytes limit is ignored since we can't check it on this stage - leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, 0, settings.read_overflow_mode_leaf); + if (settings.read_overflow_mode_leaf == OverflowMode::THROW && settings.max_rows_to_read_leaf) + leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, 0, settings.read_overflow_mode_leaf); auto process_part = [&](size_t part_index) { @@ -618,7 +618,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (!ranges.ranges.empty()) { - if (settings.read_overflow_mode == OverflowMode::THROW && (limits.max_rows || leaf_limits.max_rows)) + if (limits.max_rows || leaf_limits.max_rows) { /// Fail fast if estimated number of rows to read exceeds the limit auto current_rows_estimate = ranges.getRowsCount(); From 0ad44b821e6f8417075d26700bbea576cefd272c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Sep 2020 01:45:12 +0300 Subject: [PATCH 50/68] Fix build --- src/Parsers/formatSettingName.cpp | 1 + src/Parsers/formatSettingName.h | 1 + 2 files changed, 2 insertions(+) diff --git a/src/Parsers/formatSettingName.cpp b/src/Parsers/formatSettingName.cpp index 3f30142716e..c305496fdb3 100644 --- a/src/Parsers/formatSettingName.cpp +++ b/src/Parsers/formatSettingName.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/formatSettingName.h b/src/Parsers/formatSettingName.h index c9ed94dcc7d..a700d347a5f 100644 --- a/src/Parsers/formatSettingName.h +++ b/src/Parsers/formatSettingName.h @@ -1,5 +1,6 @@ #pragma once +#include #include From 848664c4afed4957fde0f22e88c63ac2b133742a Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 18 Sep 2020 09:34:51 +0800 Subject: [PATCH 51/68] rewrite performance test to functional test --- tests/performance/table_function_null.xml | 3 --- tests/queries/0_stateless/01493_table_function_null.reference | 0 tests/queries/0_stateless/01493_table_function_null.sql | 1 + 3 files changed, 1 insertion(+), 3 deletions(-) delete mode 100644 tests/performance/table_function_null.xml create mode 100644 tests/queries/0_stateless/01493_table_function_null.reference create mode 100644 tests/queries/0_stateless/01493_table_function_null.sql diff --git a/tests/performance/table_function_null.xml b/tests/performance/table_function_null.xml deleted file mode 100644 index 9313619d89a..00000000000 --- a/tests/performance/table_function_null.xml +++ /dev/null @@ -1,3 +0,0 @@ - - INSERT INTO function null('number UInt64') SELECT * FROM numbers_mt(1000000000); - diff --git a/tests/queries/0_stateless/01493_table_function_null.reference b/tests/queries/0_stateless/01493_table_function_null.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01493_table_function_null.sql b/tests/queries/0_stateless/01493_table_function_null.sql new file mode 100644 index 00000000000..8bf4bfdf2c2 --- /dev/null +++ b/tests/queries/0_stateless/01493_table_function_null.sql @@ -0,0 +1 @@ +INSERT INTO function null('number UInt64') SELECT * FROM numbers_mt(10000); \ No newline at end of file From fb31544d4a1f9ee9b8f6fea0130e0c02d53f9488 Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 18 Sep 2020 09:39:15 +0800 Subject: [PATCH 52/68] add blank line to the end of file --- tests/queries/0_stateless/01493_table_function_null.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01493_table_function_null.sql b/tests/queries/0_stateless/01493_table_function_null.sql index 8bf4bfdf2c2..e6b3e652b88 100644 --- a/tests/queries/0_stateless/01493_table_function_null.sql +++ b/tests/queries/0_stateless/01493_table_function_null.sql @@ -1 +1 @@ -INSERT INTO function null('number UInt64') SELECT * FROM numbers_mt(10000); \ No newline at end of file +INSERT INTO function null('number UInt64') SELECT * FROM numbers_mt(10000); From 748fb74de268849acabab2ec59c54f8d035575b9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Sep 2020 10:02:55 +0300 Subject: [PATCH 53/68] Fix build type for integration tests --- tests/ci/ci_config.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index c69ef64b807..220d8d801ec 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -431,7 +431,7 @@ }, "Integration tests (release)": { "required_build_properties": { - "compiler": "clang-11", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", From c8a5d9e0a2da0a30b2f5881a727126a2a5ac9530 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Sep 2020 10:59:26 +0300 Subject: [PATCH 54/68] Bump CI From 7f029ee03c53471937e66cebfc36e4ffa1571b72 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Fri, 18 Sep 2020 11:27:27 +0300 Subject: [PATCH 55/68] Update perf.py --- docker/test/performance-comparison/perf.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 8021c4ac880..f4992bcdc03 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -279,7 +279,7 @@ for query_index, q in enumerate(test_queries): # Try to run any query for at least the specified number of times, # before considering other stop conditions. - if run < arg.runs: + if run < args.runs: continue # For very short queries we have a special mode where we run them for at From 0520b05001d0a72d72bce9a83fcba0d9dcf2acf3 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Fri, 18 Sep 2020 12:51:44 +0300 Subject: [PATCH 56/68] Speedup wide integers (#14859) --- base/common/wide_integer.h | 6 +- base/common/wide_integer_impl.h | 656 +++++++++--------- .../01440_big_int_exotic_casts.reference | 292 ++++++++ .../01440_big_int_exotic_casts.sql | 48 ++ 4 files changed, 662 insertions(+), 340 deletions(-) create mode 100644 tests/queries/0_stateless/01440_big_int_exotic_casts.reference create mode 100644 tests/queries/0_stateless/01440_big_int_exotic_casts.sql diff --git a/base/common/wide_integer.h b/base/common/wide_integer.h index 2aeac072b3f..61d88bdcaf3 100644 --- a/base/common/wide_integer.h +++ b/base/common/wide_integer.h @@ -54,8 +54,8 @@ template class integer { public: - using base_type = uint8_t; - using signed_base_type = int8_t; + using base_type = uint64_t; + using signed_base_type = int64_t; // ctors integer() = default; @@ -127,7 +127,7 @@ private: friend class std::numeric_limits>; friend class std::numeric_limits>; - base_type m_arr[_impl::arr_size]; + base_type items[_impl::item_count]; }; template diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index 26bd6704bdc..5673ac46c4a 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -3,10 +3,6 @@ #include "throwError.h" -#ifndef CHAR_BIT -#define CHAR_BIT 8 -#endif - namespace wide { @@ -74,7 +70,7 @@ public: { using T = wide::integer; T res{}; - res.m_arr[T::_impl::big(0)] = std::numeric_limits::signed_base_type>::min(); + res.items[T::_impl::big(0)] = std::numeric_limits::signed_base_type>::min(); return res; } return 0; @@ -84,12 +80,12 @@ public: { using T = wide::integer; T res{}; - res.m_arr[T::_impl::big(0)] = is_same::value + res.items[T::_impl::big(0)] = is_same::value ? std::numeric_limits::signed_base_type>::max() : std::numeric_limits::base_type>::max(); - for (int i = 1; i < wide::integer::_impl::arr_size; ++i) + for (unsigned i = 1; i < wide::integer::_impl::item_count; ++i) { - res.m_arr[T::_impl::big(i)] = std::numeric_limits::base_type>::max(); + res.items[T::_impl::big(i)] = std::numeric_limits::base_type>::max(); } return res; } @@ -147,28 +143,45 @@ namespace wide template struct integer::_impl { - static_assert(Bits % CHAR_BIT == 0, "=)"); - - // utils - static const int base_bits = sizeof(base_type) * CHAR_BIT; - static const int arr_size = Bits / base_bits; static constexpr size_t _Bits = Bits; - static constexpr bool _is_wide_integer = true; + static constexpr const unsigned byte_count = Bits / 8; + static constexpr const unsigned item_count = byte_count / sizeof(base_type); + static constexpr const unsigned base_bits = sizeof(base_type) * 8; - // The original implementation is big-endian. We need little one. + static_assert(Bits % base_bits == 0); + + /// Simple iteration in both directions static constexpr unsigned little(unsigned idx) { return idx; } - static constexpr unsigned big(unsigned idx) { return arr_size - 1 - idx; } + static constexpr unsigned big(unsigned idx) { return item_count - 1 - idx; } static constexpr unsigned any(unsigned idx) { return idx; } + template + constexpr static bool is_negative(const T & n) noexcept + { + if constexpr (std::is_signed_v) + return n < 0; + else + return false; + } + template constexpr static bool is_negative(const integer & n) noexcept { if constexpr (std::is_same_v) - return static_cast(n.m_arr[big(0)]) < 0; + return static_cast(n.items[big(0)]) < 0; else return false; } + template + constexpr static auto make_positive(const T & n) noexcept + { + if constexpr (std::is_signed_v) + return n < 0 ? -n : n; + else + return n; + } + template constexpr static integer make_positive(const integer & n) noexcept { @@ -189,21 +202,24 @@ struct integer::_impl template constexpr static void wide_integer_from_bultin(integer & self, Integral rhs) noexcept { - auto r = _impl::to_Integral(rhs); + self.items[0] = _impl::to_Integral(rhs); + if constexpr (std::is_same_v) + self.items[1] = rhs >> base_bits; - int r_idx = 0; - for (; static_cast(r_idx) < sizeof(Integral) && r_idx < arr_size; ++r_idx) + constexpr const unsigned start = (sizeof(Integral) == 16) ? 2 : 1; + + if constexpr (std::is_signed_v) { - base_type & curr = self.m_arr[little(r_idx)]; - base_type curr_rhs = (r >> (r_idx * CHAR_BIT)) & std::numeric_limits::max(); - curr = curr_rhs; + if (rhs < 0) + { + for (unsigned i = start; i < item_count; ++i) + self.items[i] = -1; + return; + } } - for (; r_idx < arr_size; ++r_idx) - { - base_type & curr = self.m_arr[little(r_idx)]; - curr = r < 0 ? std::numeric_limits::max() : 0; - } + for (unsigned i = start; i < item_count; ++i) + self.items[i] = 0; } constexpr static void wide_integer_from_bultin(integer & self, double rhs) noexcept @@ -234,170 +250,142 @@ struct integer::_impl constexpr static void wide_integer_from_wide_integer(integer & self, const integer & rhs) noexcept { - // int Bits_to_copy = std::min(arr_size, rhs.arr_size); - auto rhs_arr_size = integer::_impl::arr_size; - int base_elems_to_copy = _impl::arr_size < rhs_arr_size ? _impl::arr_size : rhs_arr_size; - for (int i = 0; i < base_elems_to_copy; ++i) + constexpr const unsigned min_bits = (Bits < Bits2) ? Bits : Bits2; + constexpr const unsigned to_copy = min_bits / base_bits; + + for (unsigned i = 0; i < to_copy; ++i) + self.items[i] = rhs.items[i]; + + if constexpr (Bits > Bits2) { - self.m_arr[little(i)] = rhs.m_arr[little(i)]; - } - for (int i = 0; i < arr_size - base_elems_to_copy; ++i) - { - self.m_arr[big(i)] = is_negative(rhs) ? std::numeric_limits::max() : 0; + if constexpr (std::is_signed_v) + { + if (rhs < 0) + { + for (unsigned i = to_copy; i < item_count; ++i) + self.items[i] = -1; + return; + } + } + + for (unsigned i = to_copy; i < item_count; ++i) + self.items[i] = 0; } } template constexpr static bool should_keep_size() { - return sizeof(T) * CHAR_BIT <= Bits; + return sizeof(T) <= byte_count; } - constexpr static integer shift_left(const integer & rhs, int n) noexcept + constexpr static integer shift_left(const integer & rhs, unsigned n) noexcept { - if (static_cast(n) >= base_bits * arr_size) - return 0; - if (n <= 0) - return rhs; + integer lhs; + unsigned items_shift = n / base_bits; - integer lhs = rhs; - int bit_shift = n % base_bits; - unsigned n_bytes = n / base_bits; - if (bit_shift) + if (unsigned bit_shift = n % base_bits) { - lhs.m_arr[big(0)] <<= bit_shift; - for (int i = 1; i < arr_size; ++i) + unsigned overflow_shift = base_bits - bit_shift; + + lhs.items[big(0)] = rhs.items[big(items_shift)] << bit_shift; + for (unsigned i = 1; i < item_count - items_shift; ++i) { - lhs.m_arr[big(i - 1)] |= lhs.m_arr[big(i)] >> (base_bits - bit_shift); - lhs.m_arr[big(i)] <<= bit_shift; + lhs.items[big(i - 1)] |= rhs.items[big(items_shift + i)] >> overflow_shift; + lhs.items[big(i)] = rhs.items[big(items_shift + i)] << bit_shift; } } - if (n_bytes) - { - for (unsigned i = 0; i < arr_size - n_bytes; ++i) - { - lhs.m_arr[big(i)] = lhs.m_arr[big(i + n_bytes)]; - } - for (unsigned i = arr_size - n_bytes; i < arr_size; ++i) - lhs.m_arr[big(i)] = 0; - } - return lhs; - } - - constexpr static integer shift_left(const integer & rhs, int n) noexcept - { - return integer(shift_left(integer(rhs), n)); - } - - constexpr static integer shift_right(const integer & rhs, int n) noexcept - { - if (static_cast(n) >= base_bits * arr_size) - return 0; - if (n <= 0) - return rhs; - - integer lhs = rhs; - int bit_shift = n % base_bits; - unsigned n_bytes = n / base_bits; - if (bit_shift) - { - lhs.m_arr[little(0)] >>= bit_shift; - for (int i = 1; i < arr_size; ++i) - { - lhs.m_arr[little(i - 1)] |= lhs.m_arr[little(i)] << (base_bits - bit_shift); - lhs.m_arr[little(i)] >>= bit_shift; - } - } - if (n_bytes) - { - for (unsigned i = 0; i < arr_size - n_bytes; ++i) - { - lhs.m_arr[little(i)] = lhs.m_arr[little(i + n_bytes)]; - } - for (unsigned i = arr_size - n_bytes; i < arr_size; ++i) - lhs.m_arr[little(i)] = 0; - } - return lhs; - } - - constexpr static integer shift_right(const integer & rhs, int n) noexcept - { - if (static_cast(n) >= base_bits * arr_size) - return 0; - if (n <= 0) - return rhs; - - bool is_neg = is_negative(rhs); - if (!is_neg) - return shift_right(integer(rhs), n); - - integer lhs = rhs; - int bit_shift = n % base_bits; - unsigned n_bytes = n / base_bits; - if (bit_shift) - { - lhs = shift_right(integer(lhs), bit_shift); - lhs.m_arr[big(0)] |= std::numeric_limits::max() << (base_bits - bit_shift); - } - if (n_bytes) - { - for (unsigned i = 0; i < arr_size - n_bytes; ++i) - { - lhs.m_arr[little(i)] = lhs.m_arr[little(i + n_bytes)]; - } - for (unsigned i = arr_size - n_bytes; i < arr_size; ++i) - { - lhs.m_arr[little(i)] = std::numeric_limits::max(); - } - } - return lhs; - } - - template - constexpr static integer - operator_plus_T(const integer & lhs, T rhs) noexcept(std::is_same_v) - { - if (rhs < 0) - return _operator_minus_T(lhs, -rhs); else - return _operator_plus_T(lhs, rhs); + { + for (unsigned i = 0; i < item_count - items_shift; ++i) + lhs.items[big(i)] = rhs.items[big(items_shift + i)]; + } + + for (unsigned i = 0; i < items_shift; ++i) + lhs.items[little(i)] = 0; + return lhs; + } + + constexpr static integer shift_right(const integer & rhs, unsigned n) noexcept + { + integer lhs; + unsigned items_shift = n / base_bits; + unsigned bit_shift = n % base_bits; + + if (bit_shift) + { + unsigned overflow_shift = base_bits - bit_shift; + + lhs.items[little(0)] = rhs.items[little(items_shift)] >> bit_shift; + for (unsigned i = 1; i < item_count - items_shift; ++i) + { + lhs.items[little(i - 1)] |= rhs.items[little(items_shift + i)] << overflow_shift; + lhs.items[little(i)] = rhs.items[little(items_shift + i)] >> bit_shift; + } + } + else + { + for (unsigned i = 0; i < item_count - items_shift; ++i) + lhs.items[little(i)] = rhs.items[little(items_shift + i)]; + } + + if (is_negative(rhs)) + { + if (bit_shift) + lhs.items[big(items_shift)] |= std::numeric_limits::max() << (base_bits - bit_shift); + + for (unsigned i = item_count - items_shift; i < items_shift; ++i) + lhs.items[little(i)] = std::numeric_limits::max(); + } + else + { + for (unsigned i = item_count - items_shift; i < items_shift; ++i) + lhs.items[little(i)] = 0; + } + + return lhs; } private: template - constexpr static integer - _operator_minus_T(const integer & lhs, T rhs) noexcept(std::is_same_v) + constexpr static base_type get_item(const T & x, unsigned number) { - integer res = lhs; + if constexpr (IsWideInteger::value) + { + if (number < T::_impl::item_count) + return x.items[number]; + return 0; + } + else + { + if (number * sizeof(base_type) < sizeof(T)) + return x >> (number * base_bits); // & std::numeric_limits::max() + return 0; + } + } + + template + constexpr static integer + op_minus(const integer & lhs, T rhs) + { + integer res; bool is_underflow = false; - int r_idx = 0; - for (; static_cast(r_idx) < sizeof(T) && r_idx < arr_size; ++r_idx) + for (unsigned i = 0; i < item_count; ++i) { - base_type & res_i = res.m_arr[little(r_idx)]; - base_type curr_rhs = (rhs >> (r_idx * CHAR_BIT)) & std::numeric_limits::max(); + base_type lhs_item = lhs.items[little(i)]; + base_type rhs_item = get_item(rhs, i); if (is_underflow) { - --res_i; - is_underflow = res_i == std::numeric_limits::max(); + is_underflow = (lhs_item == 0); + --lhs_item; } - if (res_i < curr_rhs) + if (lhs_item < rhs_item) is_underflow = true; - res_i -= curr_rhs; - } - if (is_underflow && r_idx < arr_size) - { - --res.m_arr[little(r_idx)]; - for (int i = arr_size - 1 - r_idx - 1; i >= 0; --i) - { - if (res.m_arr[big(i + 1)] == std::numeric_limits::max()) - --res.m_arr[big(i)]; - else - break; - } + res.items[little(i)] = lhs_item - rhs_item; } return res; @@ -405,37 +393,69 @@ private: template constexpr static integer - _operator_plus_T(const integer & lhs, T rhs) noexcept(std::is_same_v) + op_plus(const integer & lhs, T rhs) { - integer res = lhs; + integer res; bool is_overflow = false; - int r_idx = 0; - for (; static_cast(r_idx) < sizeof(T) && r_idx < arr_size; ++r_idx) + for (unsigned i = 0; i < item_count; ++i) { - base_type & res_i = res.m_arr[little(r_idx)]; - base_type curr_rhs = (rhs >> (r_idx * CHAR_BIT)) & std::numeric_limits::max(); + base_type lhs_item = lhs.items[little(i)]; + base_type rhs_item = get_item(rhs, i); if (is_overflow) { - ++res_i; - is_overflow = res_i == 0; + ++lhs_item; + is_overflow = (lhs_item == 0); } - res_i += curr_rhs; - if (res_i < curr_rhs) + base_type & res_item = res.items[little(i)]; + res_item = lhs_item + rhs_item; + + if (res_item < rhs_item) is_overflow = true; } - if (is_overflow && r_idx < arr_size) + return res; + } + + template + constexpr static auto op_multiply(const integer & lhs, const T & rhs) + { + integer res{}; +#if 1 + integer lhs2 = op_plus(lhs, shift_left(lhs, 1)); + integer lhs3 = op_plus(lhs2, shift_left(lhs, 2)); +#endif + for (unsigned i = 0; i < item_count; ++i) { - ++res.m_arr[little(r_idx)]; - for (int i = arr_size - 1 - r_idx - 1; i >= 0; --i) + base_type rhs_item = get_item(rhs, i); + unsigned pos = i * base_bits; + + while (rhs_item) { - if (res.m_arr[big(i + 1)] == 0) - ++res.m_arr[big(i)]; - else - break; +#if 1 /// optimization + if ((rhs_item & 0x7) == 0x7) + { + res = op_plus(res, shift_left(lhs3, pos)); + rhs_item >>= 3; + pos += 3; + continue; + } + + if ((rhs_item & 0x3) == 0x3) + { + res = op_plus(res, shift_left(lhs2, pos)); + rhs_item >>= 2; + pos += 2; + continue; + } +#endif + if (rhs_item & 1) + res = op_plus(res, shift_left(lhs, pos)); + + rhs_item >>= 1; + ++pos; } } @@ -445,17 +465,17 @@ private: public: constexpr static integer operator_unary_tilda(const integer & lhs) noexcept { - integer res{}; + integer res; - for (int i = 0; i < arr_size; ++i) - res.m_arr[any(i)] = ~lhs.m_arr[any(i)]; + for (unsigned i = 0; i < item_count; ++i) + res.items[any(i)] = ~lhs.items[any(i)]; return res; } constexpr static integer operator_unary_minus(const integer & lhs) noexcept(std::is_same_v) { - return operator_plus_T(operator_unary_tilda(lhs), 1); + return op_plus(operator_unary_tilda(lhs), 1); } template @@ -463,15 +483,14 @@ public: { if constexpr (should_keep_size()) { - integer t = rhs; - if (is_negative(t)) - return _operator_minus_wide_integer(lhs, operator_unary_minus(t)); + if (is_negative(rhs)) + return op_minus(lhs, -rhs); else - return _operator_plus_wide_integer(lhs, t); + return op_plus(lhs, rhs); } else { - static_assert(T::_impl::_is_wide_integer, ""); + static_assert(IsWideInteger::value); return std::common_type_t, integer>::_impl::operator_plus( integer(lhs), rhs); } @@ -482,100 +501,44 @@ public: { if constexpr (should_keep_size()) { - integer t = rhs; - if (is_negative(t)) - return _operator_plus_wide_integer(lhs, operator_unary_minus(t)); + if (is_negative(rhs)) + return op_plus(lhs, -rhs); else - return _operator_minus_wide_integer(lhs, t); + return op_minus(lhs, rhs); } else { - static_assert(T::_impl::_is_wide_integer, ""); + static_assert(IsWideInteger::value); return std::common_type_t, integer>::_impl::operator_minus( integer(lhs), rhs); } } -private: - constexpr static integer _operator_minus_wide_integer( - const integer & lhs, const integer & rhs) noexcept(std::is_same_v) - { - integer res = lhs; - - bool is_underflow = false; - for (int idx = 0; idx < arr_size; ++idx) - { - base_type & res_i = res.m_arr[little(idx)]; - const base_type rhs_i = rhs.m_arr[little(idx)]; - - if (is_underflow) - { - --res_i; - is_underflow = res_i == std::numeric_limits::max(); - } - - if (res_i < rhs_i) - is_underflow = true; - - res_i -= rhs_i; - } - - return res; - } - - constexpr static integer _operator_plus_wide_integer( - const integer & lhs, const integer & rhs) noexcept(std::is_same_v) - { - integer res = lhs; - - bool is_overflow = false; - for (int idx = 0; idx < arr_size; ++idx) - { - base_type & res_i = res.m_arr[little(idx)]; - const base_type rhs_i = rhs.m_arr[little(idx)]; - - if (is_overflow) - { - ++res_i; - is_overflow = res_i == 0; - } - - res_i += rhs_i; - - if (res_i < rhs_i) - is_overflow = true; - } - - return res; - } - -public: template constexpr static auto operator_star(const integer & lhs, const T & rhs) { if constexpr (should_keep_size()) { - const integer a = make_positive(lhs); - integer t = make_positive(integer(rhs)); + integer res; - integer res = 0; - - for (size_t i = 0; i < arr_size * base_bits; ++i) + if constexpr (std::is_signed_v) { - if (t.m_arr[little(0)] & 1) - res = operator_plus(res, shift_left(a, i)); - - t = shift_right(t, 1); + res = op_multiply((is_negative(lhs) ? make_positive(lhs) : lhs), + (is_negative(rhs) ? make_positive(rhs) : rhs)); + } + else + { + res = op_multiply(lhs, (is_negative(rhs) ? make_positive(rhs) : rhs)); } - if (std::is_same_v && is_negative(integer(rhs)) != is_negative(lhs)) + if (std::is_same_v && is_negative(lhs) != is_negative(rhs)) res = operator_unary_minus(res); return res; } else { - static_assert(T::_impl::_is_wide_integer, ""); + static_assert(IsWideInteger::value); return std::common_type_t, T>::_impl::operator_star(T(lhs), rhs); } } @@ -585,25 +548,22 @@ public: { if constexpr (should_keep_size()) { - // static_assert(Signed == std::is_signed::value, - // "warning: operator_more: comparison of integers of different signs"); + if (std::numeric_limits::is_signed && (is_negative(lhs) != is_negative(rhs))) + return is_negative(rhs); - integer t = rhs; - - if (std::numeric_limits::is_signed && (is_negative(lhs) != is_negative(t))) - return is_negative(t); - - for (int i = 0; i < arr_size; ++i) + for (unsigned i = 0; i < item_count; ++i) { - if (lhs.m_arr[big(i)] != t.m_arr[big(i)]) - return lhs.m_arr[big(i)] > t.m_arr[big(i)]; + base_type rhs_item = get_item(rhs, big(i)); + + if (lhs.items[big(i)] != rhs_item) + return lhs.items[big(i)] > rhs_item; } return false; } else { - static_assert(T::_impl::_is_wide_integer, ""); + static_assert(IsWideInteger::value); return std::common_type_t, T>::_impl::operator_more(T(lhs), rhs); } } @@ -613,23 +573,22 @@ public: { if constexpr (should_keep_size()) { - // static_assert(Signed == std::is_signed::value, - // "warning: operator_less: comparison of integers of different signs"); - - integer t = rhs; - - if (std::numeric_limits::is_signed && (is_negative(lhs) != is_negative(t))) + if (std::numeric_limits::is_signed && (is_negative(lhs) != is_negative(rhs))) return is_negative(lhs); - for (int i = 0; i < arr_size; ++i) - if (lhs.m_arr[big(i)] != t.m_arr[big(i)]) - return lhs.m_arr[big(i)] < t.m_arr[big(i)]; + for (unsigned i = 0; i < item_count; ++i) + { + base_type rhs_item = get_item(rhs, big(i)); + + if (lhs.items[big(i)] != rhs_item) + return lhs.items[big(i)] < rhs_item; + } return false; } else { - static_assert(T::_impl::_is_wide_integer, ""); + static_assert(IsWideInteger::value); return std::common_type_t, T>::_impl::operator_less(T(lhs), rhs); } } @@ -639,17 +598,19 @@ public: { if constexpr (should_keep_size()) { - integer t = rhs; + for (unsigned i = 0; i < item_count; ++i) + { + base_type rhs_item = get_item(rhs, any(i)); - for (int i = 0; i < arr_size; ++i) - if (lhs.m_arr[any(i)] != t.m_arr[any(i)]) + if (lhs.items[any(i)] != rhs_item) return false; + } return true; } else { - static_assert(T::_impl::_is_wide_integer, ""); + static_assert(IsWideInteger::value); return std::common_type_t, T>::_impl::operator_eq(T(lhs), rhs); } } @@ -659,16 +620,15 @@ public: { if constexpr (should_keep_size()) { - integer t = rhs; - integer res = lhs; + integer res; - for (int i = 0; i < arr_size; ++i) - res.m_arr[any(i)] |= t.m_arr[any(i)]; + for (unsigned i = 0; i < item_count; ++i) + res.items[little(i)] = lhs.items[little(i)] | get_item(rhs, i); return res; } else { - static_assert(T::_impl::_is_wide_integer, ""); + static_assert(IsWideInteger::value); return std::common_type_t, T>::_impl::operator_pipe(T(lhs), rhs); } } @@ -678,43 +638,48 @@ public: { if constexpr (should_keep_size()) { - integer t = rhs; - integer res = lhs; + integer res; - for (int i = 0; i < arr_size; ++i) - res.m_arr[any(i)] &= t.m_arr[any(i)]; + for (unsigned i = 0; i < item_count; ++i) + res.items[little(i)] = lhs.items[little(i)] & get_item(rhs, i); return res; } else { - static_assert(T::_impl::_is_wide_integer, ""); + static_assert(IsWideInteger::value); return std::common_type_t, T>::_impl::operator_amp(T(lhs), rhs); } } private: template - constexpr static void divide(const T & lhserator, const T & denominator, T & quotient, T & remainder) + constexpr static bool is_zero(const T & x) { bool is_zero = true; - for (auto c : denominator.m_arr) + for (auto item : x.items) { - if (c != 0) + if (item != 0) { is_zero = false; break; } } + return is_zero; + } - if (is_zero) + /// returns quotient as result and remainder in numerator. + template + constexpr static T divide(T & numerator, T && denominator) + { + if (is_zero(denominator)) throwError("divide by zero"); - T n = lhserator; - T d = denominator; + T & n = numerator; + T & d = denominator; T x = 1; - T answer = 0; + T quotient = 0; - while (!operator_more(d, n) && operator_eq(operator_amp(shift_right(d, base_bits * arr_size - 1), 1), 0)) + while (!operator_more(d, n) && operator_eq(operator_amp(shift_right(d, base_bits * item_count - 1), 1), 0)) { x = shift_left(x, 1); d = shift_left(d, 1); @@ -725,15 +690,14 @@ private: if (!operator_more(d, n)) { n = operator_minus(n, d); - answer = operator_pipe(answer, x); + quotient = operator_pipe(quotient, x); } x = shift_right(x, 1); d = shift_right(d, 1); } - quotient = answer; - remainder = n; + return quotient; } public: @@ -742,18 +706,16 @@ public: { if constexpr (should_keep_size()) { - integer o = rhs; - integer quotient{}, remainder{}; - divide(make_positive(lhs), make_positive(o), quotient, remainder); + integer numerator = make_positive(lhs); + integer quotient = divide(numerator, make_positive(integer(rhs))); - if (std::is_same_v && is_negative(o) != is_negative(lhs)) + if (std::is_same_v && is_negative(rhs) != is_negative(lhs)) quotient = operator_unary_minus(quotient); - return quotient; } else { - static_assert(T::_impl::_is_wide_integer, ""); + static_assert(IsWideInteger::value); return std::common_type_t, integer>::operator_slash(T(lhs), rhs); } } @@ -763,18 +725,16 @@ public: { if constexpr (should_keep_size()) { - integer o = rhs; - integer quotient{}, remainder{}; - divide(make_positive(lhs), make_positive(o), quotient, remainder); + integer remainder = make_positive(lhs); + divide(remainder, make_positive(integer(rhs))); if (std::is_same_v && is_negative(lhs)) remainder = operator_unary_minus(remainder); - return remainder; } else { - static_assert(T::_impl::_is_wide_integer, ""); + static_assert(IsWideInteger::value); return std::common_type_t, integer>::operator_percent(T(lhs), rhs); } } @@ -788,13 +748,13 @@ public: integer t(rhs); integer res = lhs; - for (int i = 0; i < arr_size; ++i) - res.m_arr[any(i)] ^= t.m_arr[any(i)]; + for (unsigned i = 0; i < item_count; ++i) + res.items[any(i)] ^= t.items[any(i)]; return res; } else { - static_assert(T::_impl::_is_wide_integer, ""); + static_assert(IsWideInteger::value); return T::operator_circumflex(T(lhs), rhs); } } @@ -815,20 +775,20 @@ public: { if (*c >= '0' && *c <= '9') { - res = operator_star(res, 16U); - res = operator_plus_T(res, *c - '0'); + res = op_multiply(res, 16U); + res = op_plus(res, *c - '0'); ++c; } else if (*c >= 'a' && *c <= 'f') { - res = operator_star(res, 16U); - res = operator_plus_T(res, *c - 'a' + 10U); + res = op_multiply(res, 16U); + res = op_plus(res, *c - 'a' + 10U); ++c; } else if (*c >= 'A' && *c <= 'F') { // tolower must be used, but it is not constexpr - res = operator_star(res, 16U); - res = operator_plus_T(res, *c - 'A' + 10U); + res = op_multiply(res, 16U); + res = op_plus(res, *c - 'A' + 10U); ++c; } else @@ -842,8 +802,8 @@ public: if (*c < '0' || *c > '9') throwError("invalid char from"); - res = operator_star(res, 10U); - res = operator_plus_T(res, *c - '0'); + res = op_multiply(res, 10U); + res = op_plus(res, *c - '0'); ++c; } } @@ -860,7 +820,7 @@ public: template template constexpr integer::integer(T rhs) noexcept - : m_arr{} + : items{} { if constexpr (IsWideInteger::value) _impl::wide_integer_from_wide_integer(*this, rhs); @@ -871,7 +831,7 @@ constexpr integer::integer(T rhs) noexcept template template constexpr integer::integer(std::initializer_list il) noexcept - : m_arr{} + : items{} { if (il.size() == 1) { @@ -967,14 +927,25 @@ constexpr integer & integer::operator^=(const T & rh template constexpr integer & integer::operator<<=(int n) noexcept { - *this = _impl::shift_left(*this, n); + if (static_cast(n) >= Bits) + *this = 0; + else if (n > 0) + *this = _impl::shift_left(*this, n); return *this; } template constexpr integer & integer::operator>>=(int n) noexcept { - *this = _impl::shift_right(*this, n); + if (static_cast(n) >= Bits) + { + if (is_negative(*this)) + *this = -1; + else + *this = 0; + } + else if (n > 0) + *this = _impl::shift_right(*this, n); return *this; } @@ -1018,13 +989,16 @@ template template constexpr integer::operator T() const noexcept { - static_assert(std::numeric_limits::is_integer, ""); - T res = 0; - for (size_t r_idx = 0; r_idx < _impl::arr_size && r_idx < sizeof(T); ++r_idx) + if constexpr (std::is_same_v) { - res |= (T(m_arr[_impl::little(r_idx)]) << (_impl::base_bits * r_idx)); + static_assert(Bits >= 128); + return (__int128(items[1]) << 64) | items[0]; + } + else + { + static_assert(std::numeric_limits::is_integer); + return items[0]; } - return res; } template @@ -1038,12 +1012,12 @@ constexpr integer::operator long double() const noexcept tmp = -tmp; long double res = 0; - for (size_t idx = 0; idx < _impl::arr_size; ++idx) + for (unsigned i = 0; i < _impl::item_count; ++i) { long double t = res; res *= std::numeric_limits::max(); res += t; - res += tmp.m_arr[_impl::big(idx)]; + res += tmp.items[_impl::big(i)]; } if (_impl::is_negative(*this)) @@ -1187,11 +1161,19 @@ std::common_type_t constexpr operator^(const Integral & lhs template constexpr integer operator<<(const integer & lhs, int n) noexcept { + if (static_cast(n) >= Bits) + return 0; + if (n <= 0) + return lhs; return integer::_impl::shift_left(lhs, n); } template constexpr integer operator>>(const integer & lhs, int n) noexcept { + if (static_cast(n) >= Bits) + return 0; + if (n <= 0) + return lhs; return integer::_impl::shift_right(lhs, n); } @@ -1277,7 +1259,7 @@ struct hash> { static_assert(Bits % (sizeof(size_t) * 8) == 0); - const auto * ptr = reinterpret_cast(lhs.m_arr); + const auto * ptr = reinterpret_cast(lhs.items); unsigned count = Bits / (sizeof(size_t) * 8); size_t res = 0; diff --git a/tests/queries/0_stateless/01440_big_int_exotic_casts.reference b/tests/queries/0_stateless/01440_big_int_exotic_casts.reference new file mode 100644 index 00000000000..fb3cdce7ad9 --- /dev/null +++ b/tests/queries/0_stateless/01440_big_int_exotic_casts.reference @@ -0,0 +1,292 @@ +0 0.0 0.00 0.000000 0.0000000 +1 1.0 1.00 1.000000 1.0000000 +8 8.0 8.00 8.000000 8.0000000 +27 27.0 27.00 27.000000 27.0000000 +64 64.0 64.00 64.000000 64.0000000 +125 125.0 125.00 125.000000 125.0000000 +216 216.0 216.00 216.000000 216.0000000 +343 343.0 343.00 343.000000 343.0000000 +512 512.0 512.00 512.000000 512.0000000 +729 729.0 729.00 729.000000 729.0000000 +0 0.0 0.00 0.000000 0.0000000 +1 1.0 1.00 1.000000 1.0000000 +8 8.0 8.00 8.000000 8.0000000 +27 27.0 27.00 27.000000 27.0000000 +64 64.0 64.00 64.000000 64.0000000 +125 125.0 125.00 125.000000 125.0000000 +216 216.0 216.00 216.000000 216.0000000 +343 343.0 343.00 343.000000 343.0000000 +512 512.0 512.00 512.000000 512.0000000 +729 729.0 729.00 729.000000 729.0000000 +0 0.0 0.00 0.000000 0.0000000 +1 1.0 1.00 1.000000 1.0000000 +8 8.0 8.00 8.000000 8.0000000 +27 27.0 27.00 27.000000 27.0000000 +64 64.0 64.00 64.000000 64.0000000 +125 125.0 125.00 125.000000 125.0000000 +216 216.0 216.00 216.000000 216.0000000 +343 343.0 343.00 343.000000 343.0000000 +512 512.0 512.00 512.000000 512.0000000 +729 729.0 729.00 729.000000 729.0000000 +0 0.0 0.00 0.000000 0.0000000 +1 1.0 1.00 1.000000 1.0000000 +8 8.0 8.00 8.000000 8.0000000 +27 27.0 27.00 27.000000 27.0000000 +64 64.0 64.00 64.000000 64.0000000 +125 125.0 125.00 125.000000 125.0000000 +216 216.0 216.00 216.000000 216.0000000 +343 343.0 343.00 343.000000 343.0000000 +512 512.0 512.00 512.000000 512.0000000 +729 729.0 729.00 729.000000 729.0000000 +0 0.0 0.00 0.000000 0.0000000 +1 1.0 1.00 1.000000 1.0000000 +8 8.0 8.00 8.000000 8.0000000 +27 27.0 27.00 27.000000 27.0000000 +64 64.0 64.00 64.000000 64.0000000 +125 125.0 125.00 125.000000 125.0000000 +216 216.0 216.00 216.000000 216.0000000 +343 343.0 343.00 343.000000 343.0000000 +512 512.0 512.00 512.000000 512.0000000 +729 729.0 729.00 729.000000 729.0000000 +0 0.0 0.00 0.000000 0.0000000 +1 1.0 1.00 1.000000 1.0000000 +8 8.0 8.00 8.000000 8.0000000 +27 27.0 27.00 27.000000 27.0000000 +64 64.0 64.00 64.000000 64.0000000 +125 125.0 125.00 125.000000 125.0000000 +216 216.0 216.00 216.000000 216.0000000 +343 343.0 343.00 343.000000 343.0000000 +512 512.0 512.00 512.000000 512.0000000 +729 729.0 729.00 729.000000 729.0000000 +0 0.0 0.00 0.000000 0.0000000 +1 1.0 1.00 1.000000 1.0000000 +8 8.0 8.00 8.000000 8.0000000 +27 27.0 27.00 27.000000 27.0000000 +64 64.0 64.00 64.000000 64.0000000 +125 125.0 125.00 125.000000 125.0000000 +216 216.0 216.00 216.000000 216.0000000 +343 343.0 343.00 343.000000 343.0000000 +512 512.0 512.00 512.000000 512.0000000 +729 729.0 729.00 729.000000 729.0000000 +0 0.0 0.00 0.000000 0.0000000 +1 1.0 1.00 1.000000 1.0000000 +8 8.0 8.00 8.000000 8.0000000 +27 27.0 27.00 27.000000 27.0000000 +64 64.0 64.00 64.000000 64.0000000 +125 125.0 125.00 125.000000 125.0000000 +216 216.0 216.00 216.000000 216.0000000 +343 343.0 343.00 343.000000 343.0000000 +512 512.0 512.00 512.000000 512.0000000 +729 729.0 729.00 729.000000 729.0000000 +0 0.0 0.00 0.000000 0.0000000 +1 1.0 1.00 1.000000 1.0000000 +8 8.0 8.00 8.000000 8.0000000 +27 27.0 27.00 27.000000 27.0000000 +64 64.0 64.00 64.000000 64.0000000 +125 125.0 125.00 125.000000 125.0000000 +216 216.0 216.00 216.000000 216.0000000 +343 343.0 343.00 343.000000 343.0000000 +512 512.0 512.00 512.000000 512.0000000 +729 729.0 729.00 729.000000 729.0000000 +0 0.0 0.00 0.000000 0.0000000 +-1 -1.0 -1.00 -1.000000 -1.0000000 +-4 -4.0 -4.00 -4.000000 -4.0000000 +-9 -9.0 -9.00 -9.000000 -9.0000000 +-16 -16.0 -16.00 -16.000000 -16.0000000 +-25 -25.0 -25.00 -25.000000 -25.0000000 +-36 -36.0 -36.00 -36.000000 -36.0000000 +-49 -49.0 -49.00 -49.000000 -49.0000000 +-64 -64.0 -64.00 -64.000000 -64.0000000 +-81 -81.0 -81.00 -81.000000 -81.0000000 +0 0.0 0.00 0.000000 0.0000000 +-1 -1.0 -1.00 -1.000000 -1.0000000 +-4 -4.0 -4.00 -4.000000 -4.0000000 +-9 -9.0 -9.00 -9.000000 -9.0000000 +-16 -16.0 -16.00 -16.000000 -16.0000000 +-25 -25.0 -25.00 -25.000000 -25.0000000 +-36 -36.0 -36.00 -36.000000 -36.0000000 +-49 -49.0 -49.00 -49.000000 -49.0000000 +-64 -64.0 -64.00 -64.000000 -64.0000000 +-81 -81.0 -81.00 -81.000000 -81.0000000 +0 0.0 0.00 0.000000 0.0000000 +-1 -1.0 -1.00 -1.000000 -1.0000000 +-4 -4.0 -4.00 -4.000000 -4.0000000 +-9 -9.0 -9.00 -9.000000 -9.0000000 +-16 -16.0 -16.00 -16.000000 -16.0000000 +-25 -25.0 -25.00 -25.000000 -25.0000000 +-36 -36.0 -36.00 -36.000000 -36.0000000 +-49 -49.0 -49.00 -49.000000 -49.0000000 +-64 -64.0 -64.00 -64.000000 -64.0000000 +-81 -81.0 -81.00 -81.000000 -81.0000000 +0 0.0 0.00 0.000000 0.0000000 +-1 -1.0 -1.00 -1.000000 -1.0000000 +-4 -4.0 -4.00 -4.000000 -4.0000000 +-9 -9.0 -9.00 -9.000000 -9.0000000 +-16 -16.0 -16.00 -16.000000 -16.0000000 +-25 -25.0 -25.00 -25.000000 -25.0000000 +-36 -36.0 -36.00 -36.000000 -36.0000000 +-49 -49.0 -49.00 -49.000000 -49.0000000 +-64 -64.0 -64.00 -64.000000 -64.0000000 +-81 -81.0 -81.00 -81.000000 -81.0000000 +0 0.0 0.00 0.000000 0.0000000 +-1 -1.0 -1.00 -1.000000 -1.0000000 +-4 -4.0 -4.00 -4.000000 -4.0000000 +-9 -9.0 -9.00 -9.000000 -9.0000000 +-16 -16.0 -16.00 -16.000000 -16.0000000 +-25 -25.0 -25.00 -25.000000 -25.0000000 +-36 -36.0 -36.00 -36.000000 -36.0000000 +-49 -49.0 -49.00 -49.000000 -49.0000000 +-64 -64.0 -64.00 -64.000000 -64.0000000 +-81 -81.0 -81.00 -81.000000 -81.0000000 +0 0.0 0.00 0.000000 0.0000000 +-1 -1.0 -1.00 -1.000000 -1.0000000 +-4 -4.0 -4.00 -4.000000 -4.0000000 +-9 -9.0 -9.00 -9.000000 -9.0000000 +-16 -16.0 -16.00 -16.000000 -16.0000000 +-25 -25.0 -25.00 -25.000000 -25.0000000 +-36 -36.0 -36.00 -36.000000 -36.0000000 +-49 -49.0 -49.00 -49.000000 -49.0000000 +-64 -64.0 -64.00 -64.000000 -64.0000000 +-81 -81.0 -81.00 -81.000000 -81.0000000 +0 0.0 0.00 0.000000 0.0000000 +-1 -1.0 -1.00 -1.000000 -1.0000000 +-4 -4.0 -4.00 -4.000000 -4.0000000 +-9 -9.0 -9.00 -9.000000 -9.0000000 +-16 -16.0 -16.00 -16.000000 -16.0000000 +-25 -25.0 -25.00 -25.000000 -25.0000000 +-36 -36.0 -36.00 -36.000000 -36.0000000 +-49 -49.0 -49.00 -49.000000 -49.0000000 +-64 -64.0 -64.00 -64.000000 -64.0000000 +-81 -81.0 -81.00 -81.000000 -81.0000000 +-0 0.0 0.00 0.000000 0.0000000 +-1 -1.0 -1.00 -1.000000 -1.0000000 +-4 -4.0 -4.00 -4.000000 -4.0000000 +-9 -9.0 -9.00 -9.000000 -9.0000000 +-16 -16.0 -16.00 -16.000000 -16.0000000 +-25 -25.0 -25.00 -25.000000 -25.0000000 +-36 -36.0 -36.00 -36.000000 -36.0000000 +-49 -49.0 -49.00 -49.000000 -49.0000000 +-64 -64.0 -64.00 -64.000000 -64.0000000 +-81 -81.0 -81.00 -81.000000 -81.0000000 +-0 0.0 0.00 0.000000 0.0000000 +-1 -1.0 -1.00 -1.000000 -1.0000000 +-4 -4.0 -4.00 -4.000000 -4.0000000 +-9 -9.0 -9.00 -9.000000 -9.0000000 +-16 -16.0 -16.00 -16.000000 -16.0000000 +-25 -25.0 -25.00 -25.000000 -25.0000000 +-36 -36.0 -36.00 -36.000000 -36.0000000 +-49 -49.0 -49.00 -49.000000 -49.0000000 +-64 -64.0 -64.00 -64.000000 -64.0000000 +-81 -81.0 -81.00 -81.000000 -81.0000000 +0 0 0 0 +4294967295 4294967295 4294967295 4294967295 +8589934588 8589934588 8589934588 8589934588 +12884901879 12884901879 12884901879 12884901879 +17179869168 17179869168 17179869168 17179869168 +21474836455 21474836455 21474836455 21474836455 +25769803740 25769803740 25769803740 25769803740 +30064771023 30064771023 30064771023 30064771023 +34359738304 34359738304 34359738304 34359738304 +38654705583 38654705583 38654705583 38654705583 +0 0 0 0 +18446744073709551615 18446744073709551615 18446744073709551615 18446744073709551615 +18446744073709551612 18446744073709551612 18446744073709551612 18446744073709551612 +18446744073709551607 18446744073709551607 18446744073709551607 18446744073709551607 +18446744073709551600 18446744073709551600 18446744073709551600 18446744073709551600 +18446744073709551591 18446744073709551591 18446744073709551591 18446744073709551591 +18446744073709551580 18446744073709551580 18446744073709551580 18446744073709551580 +18446744073709551567 18446744073709551567 18446744073709551567 18446744073709551567 +18446744073709551552 18446744073709551552 18446744073709551552 18446744073709551552 +18446744073709551535 18446744073709551535 18446744073709551535 18446744073709551535 +0 0 0 0 +115792089237316195423570985008687907853269984665640564039457584007913129639935 -1 -1 115792089237316195423570985008687907853269984665640564039457584007913129639935 +115792089237316195423570985008687907853269984665640564039457584007913129639932 -4 -4 115792089237316195423570985008687907853269984665640564039457584007913129639932 +115792089237316195423570985008687907853269984665640564039457584007913129639927 -9 -9 115792089237316195423570985008687907853269984665640564039457584007913129639927 +115792089237316195423570985008687907853269984665640564039457584007913129639920 -16 -16 115792089237316195423570985008687907853269984665640564039457584007913129639920 +115792089237316195423570985008687907853269984665640564039457584007913129639911 -25 -25 115792089237316195423570985008687907853269984665640564039457584007913129639911 +115792089237316195423570985008687907853269984665640564039457584007913129639900 -36 -36 115792089237316195423570985008687907853269984665640564039457584007913129639900 +115792089237316195423570985008687907853269984665640564039457584007913129639887 -49 -49 115792089237316195423570985008687907853269984665640564039457584007913129639887 +115792089237316195423570985008687907853269984665640564039457584007913129639872 -64 -64 115792089237316195423570985008687907853269984665640564039457584007913129639872 +115792089237316195423570985008687907853269984665640564039457584007913129639855 -81 -81 115792089237316195423570985008687907853269984665640564039457584007913129639855 +0 0 0 0 +-1 -1 -1 115792089237316195423570985008687907853269984665640564039457584007913129639935 +-4 -4 -4 115792089237316195423570985008687907853269984665640564039457584007913129639932 +-9 -9 -9 115792089237316195423570985008687907853269984665640564039457584007913129639927 +-16 -16 -16 115792089237316195423570985008687907853269984665640564039457584007913129639920 +-25 -25 -25 115792089237316195423570985008687907853269984665640564039457584007913129639911 +-36 -36 -36 115792089237316195423570985008687907853269984665640564039457584007913129639900 +-49 -49 -49 115792089237316195423570985008687907853269984665640564039457584007913129639887 +-64 -64 -64 115792089237316195423570985008687907853269984665640564039457584007913129639872 +-81 -81 -81 115792089237316195423570985008687907853269984665640564039457584007913129639855 +0 0 0 0 +-1 -1 -1 115792089237316195423570985008687907853269984665640564039457584007913129639935 +-4 -4 -4 115792089237316195423570985008687907853269984665640564039457584007913129639932 +-9 -9 -9 115792089237316195423570985008687907853269984665640564039457584007913129639927 +-16 -16 -16 115792089237316195423570985008687907853269984665640564039457584007913129639920 +-25 -25 -25 115792089237316195423570985008687907853269984665640564039457584007913129639911 +-36 -36 -36 115792089237316195423570985008687907853269984665640564039457584007913129639900 +-49 -49 -49 115792089237316195423570985008687907853269984665640564039457584007913129639887 +-64 -64 -64 115792089237316195423570985008687907853269984665640564039457584007913129639872 +-81 -81 -81 115792089237316195423570985008687907853269984665640564039457584007913129639855 +0 0 0 0 +-1 -1 -1 115792089237316195423570985008687907853269984665640564039457584007913129639935 +-4 -4 -4 115792089237316195423570985008687907853269984665640564039457584007913129639932 +-9 -9 -9 115792089237316195423570985008687907853269984665640564039457584007913129639927 +-16 -16 -16 115792089237316195423570985008687907853269984665640564039457584007913129639920 +-25 -25 -25 115792089237316195423570985008687907853269984665640564039457584007913129639911 +-36 -36 -36 115792089237316195423570985008687907853269984665640564039457584007913129639900 +-49 -49 -49 115792089237316195423570985008687907853269984665640564039457584007913129639887 +-64 -64 -64 115792089237316195423570985008687907853269984665640564039457584007913129639872 +-81 -81 -81 115792089237316195423570985008687907853269984665640564039457584007913129639855 +0 0 0 0 +-1 -1 -1 115792089237316195423570985008687907853269984665640564039457584007913129639935 +-4 -4 -4 115792089237316195423570985008687907853269984665640564039457584007913129639932 +-9 -9 -9 115792089237316195423570985008687907853269984665640564039457584007913129639927 +-16 -16 -16 115792089237316195423570985008687907853269984665640564039457584007913129639920 +-25 -25 -25 115792089237316195423570985008687907853269984665640564039457584007913129639911 +-36 -36 -36 115792089237316195423570985008687907853269984665640564039457584007913129639900 +-49 -49 -49 115792089237316195423570985008687907853269984665640564039457584007913129639887 +-64 -64 -64 115792089237316195423570985008687907853269984665640564039457584007913129639872 +-81 -81 -81 115792089237316195423570985008687907853269984665640564039457584007913129639855 +0 0 0 0 +-1 -1 -1 115792089237316195423570985008687907853269984665640564039457584007913129639935 +-4 -4 -4 115792089237316195423570985008687907853269984665640564039457584007913129639932 +-9 -9 -9 115792089237316195423570985008687907853269984665640564039457584007913129639927 +-16 -16 -16 115792089237316195423570985008687907853269984665640564039457584007913129639920 +-25 -25 -25 115792089237316195423570985008687907853269984665640564039457584007913129639911 +-36 -36 -36 115792089237316195423570985008687907853269984665640564039457584007913129639900 +-49 -49 -49 115792089237316195423570985008687907853269984665640564039457584007913129639887 +-64 -64 -64 115792089237316195423570985008687907853269984665640564039457584007913129639872 +-81 -81 -81 115792089237316195423570985008687907853269984665640564039457584007913129639855 +0 0 0 0 +-1 -1 -1 115792089237316195423570985008687907853269984665640564039457584007913129639935 +-4 -4 -4 115792089237316195423570985008687907853269984665640564039457584007913129639932 +-9 -9 -9 115792089237316195423570985008687907853269984665640564039457584007913129639927 +-16 -16 -16 115792089237316195423570985008687907853269984665640564039457584007913129639920 +-25 -25 -25 115792089237316195423570985008687907853269984665640564039457584007913129639911 +-36 -36 -36 115792089237316195423570985008687907853269984665640564039457584007913129639900 +-49 -49 -49 115792089237316195423570985008687907853269984665640564039457584007913129639887 +-64 -64 -64 115792089237316195423570985008687907853269984665640564039457584007913129639872 +-81 -81 -81 115792089237316195423570985008687907853269984665640564039457584007913129639855 +0 0 0 0 +1 0 0 0 +2 0 0 0 +3 0 0 0 +4 0 0 0 +5 0 0 0 +6 0 0 0 +7 0 0 0 +8 0 0 0 +9 0 0 0 +0 0 0 0 +-1 0 0 0 +-2 0 0 0 +-3 0 0 0 +-4 0 0 0 +-5 0 0 0 +-6 0 0 0 +-7 0 0 0 +-8 0 0 0 +-9 0 0 0 +2499500025000000 2499500025000000 2499500025000000 2499500025000000.00 +0 0 0 0.00 diff --git a/tests/queries/0_stateless/01440_big_int_exotic_casts.sql b/tests/queries/0_stateless/01440_big_int_exotic_casts.sql new file mode 100644 index 00000000000..42fde9da01b --- /dev/null +++ b/tests/queries/0_stateless/01440_big_int_exotic_casts.sql @@ -0,0 +1,48 @@ +SELECT toUInt32(number * number) * number y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; +SELECT toUInt64(number * number) * number y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; +SELECT toUInt256(number * number) * number y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; +SELECT toInt32(number * number) * number y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; +SELECT toInt64(number * number) * number y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; +SELECT toInt128(number * number) * number y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; +SELECT toInt256(number * number) * number y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; +SELECT toFloat32(number * number) * number y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; +SELECT toFloat64(number * number) * number y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; + +SELECT toUInt32(number * number) * -1 y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; +SELECT toUInt64(number * number) * -1 y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; +SELECT toUInt256(number * number) * -1 y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; +SELECT toInt32(number * number) * -1 y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; +SELECT toInt64(number * number) * -1 y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; +SELECT toInt128(number * number) * -1 y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; +SELECT toInt256(number * number) * -1 y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; +SELECT toFloat32(number * number) * -1 y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; +SELECT toFloat64(number * number) * -1 y, toDecimal32(y, 1), toDecimal64(y, 2), toDecimal128(y, 6), toDecimal256(y, 7) FROM numbers_mt(10) ORDER BY number; + +SELECT toUInt32(number * -1) * number y, toInt128(y), toInt256(y), toUInt256(y) FROM numbers_mt(10) ORDER BY number; +SELECT toUInt64(number * -1) * number y, toInt128(y), toInt256(y), toUInt256(y) FROM numbers_mt(10) ORDER BY number; +SELECT toUInt256(number * -1) * number y, toInt128(y), toInt256(y), toUInt256(y) FROM numbers_mt(10) ORDER BY number; +SELECT toInt32(number * -1) * number y, toInt128(y), toInt256(y), toUInt256(y) FROM numbers_mt(10) ORDER BY number; +SELECT toInt64(number * -1) * number y, toInt128(y), toInt256(y), toUInt256(y) FROM numbers_mt(10) ORDER BY number; +SELECT toInt128(number * -1) * number y, toInt128(y), toInt256(y), toUInt256(y) FROM numbers_mt(10) ORDER BY number; +SELECT toInt256(number * -1) * number y, toInt128(y), toInt256(y), toUInt256(y) FROM numbers_mt(10) ORDER BY number; +SELECT toFloat32(number * -1) * number y, toInt128(y), toInt256(y), toUInt256(y) FROM numbers_mt(10) ORDER BY number; +SELECT toFloat64(number * -1) * number y, toInt128(y), toInt256(y), toUInt256(y) FROM numbers_mt(10) ORDER BY number; + +SELECT number y, toInt128(number) - y, toInt256(number) - y, toUInt256(number) - y FROM numbers_mt(10) ORDER BY number; +SELECT -number y, toInt128(number) + y, toInt256(number) + y, toUInt256(number) + y FROM numbers_mt(10) ORDER BY number; + + +SET allow_experimental_bigint_types = 1; + +DROP TABLE IF EXISTS t; +CREATE TABLE t (x UInt64, i256 Int256, u256 UInt256, d256 Decimal256(2)) ENGINE = Memory; + +INSERT INTO t SELECT number * number * number AS x, x AS i256, x AS u256, x AS d256 FROM numbers(10000); + +SELECT sum(x), sum(i256), sum(u256), sum(d256) FROM t; + +INSERT INTO t SELECT -number * number * number AS x, x AS i256, x AS u256, x AS d256 FROM numbers(10000); + +SELECT sum(x), sum(i256), sum(u256), sum(d256) FROM t; + +DROP TABLE t; From e1cf60990a9d4801a184c4a7b5561016a19f23fd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 18 Sep 2020 14:06:30 +0300 Subject: [PATCH 57/68] minor fixes --- src/TableFunctions/TableFunctionNull.cpp | 6 ++---- src/TableFunctions/ya.make | 1 + 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/TableFunctions/TableFunctionNull.cpp b/src/TableFunctions/TableFunctionNull.cpp index d3fccb3d385..38ed8f2475c 100644 --- a/src/TableFunctions/TableFunctionNull.cpp +++ b/src/TableFunctions/TableFunctionNull.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include "registerTableFunctions.h" @@ -26,10 +27,7 @@ StoragePtr TableFunctionNull::executeImpl(const ASTPtr & ast_function, const Con if (arguments.size() != 1) throw Exception("Table function '" + getName() + "' requires 'structure'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - const auto * literal = arguments[0]->as(); - if (!literal) - throw Exception("Table function " + getName() + " requested literal argument.", ErrorCodes::LOGICAL_ERROR); - auto structure = literal->value.safeGet(); + auto structure = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[0], context)->as()->value.safeGet(); ColumnsDescription columns = parseColumnsListFromString(structure, context); auto res = StorageNull::create(StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription()); diff --git a/src/TableFunctions/ya.make b/src/TableFunctions/ya.make index c9c80003ffb..03432e2bbbc 100644 --- a/src/TableFunctions/ya.make +++ b/src/TableFunctions/ya.make @@ -19,6 +19,7 @@ SRCS( TableFunctionInput.cpp TableFunctionMerge.cpp TableFunctionMySQL.cpp + TableFunctionNull.cpp TableFunctionNumbers.cpp TableFunctionRemote.cpp TableFunctionURL.cpp From 1f54fdfb7c8ec4594ca6d7f7687ea20e353514d3 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 18 Sep 2020 14:52:30 +0300 Subject: [PATCH 58/68] Update TableFunctionNull.cpp --- src/TableFunctions/TableFunctionNull.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionNull.cpp b/src/TableFunctions/TableFunctionNull.cpp index 38ed8f2475c..00283554041 100644 --- a/src/TableFunctions/TableFunctionNull.cpp +++ b/src/TableFunctions/TableFunctionNull.cpp @@ -15,7 +15,6 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int LOGICAL_ERROR; } StoragePtr TableFunctionNull::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const From 1c674cea93678a21f640da4fa243f46b348132c0 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 18 Sep 2020 15:26:48 +0300 Subject: [PATCH 59/68] Update version_date.tsv after release 20.8.3.18 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 3915000cec3..89413bb065a 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v20.8.3.18-stable 2020-09-18 v20.8.2.3-stable 2020-09-08 v20.7.2.30-stable 2020-08-31 v20.6.6.7-stable 2020-09-11 From 10a288f89642f9fdc1bcbe14f9aa01e39f336491 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 18 Sep 2020 15:46:35 +0300 Subject: [PATCH 60/68] Update version_date.tsv after release 20.7.3.7 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 89413bb065a..848de35762d 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,5 +1,6 @@ v20.8.3.18-stable 2020-09-18 v20.8.2.3-stable 2020-09-08 +v20.7.3.7-stable 2020-09-18 v20.7.2.30-stable 2020-08-31 v20.6.6.7-stable 2020-09-11 v20.6.5.8-stable 2020-09-03 From e5dfc38bfe0918cd5a7500bee2308e8c10ff274e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 18 Sep 2020 16:13:04 +0300 Subject: [PATCH 61/68] Skip 01455_shard_leaf_max_rows_bytes_to_read for arcadia. --- tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 123b4c3d144..d776f8d80d6 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -143,3 +143,4 @@ 01474_bad_global_join 01473_event_time_microseconds 01461_query_start_time_microseconds +01455_shard_leaf_max_rows_bytes_to_read From 0052bbdd84b2b4b1e0cda7eac67f4bbd1accc0e3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 18 Sep 2020 16:19:23 +0300 Subject: [PATCH 62/68] Update version_date.tsv after release 20.6.7.4 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 848de35762d..9c5f062eade 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -2,6 +2,7 @@ v20.8.3.18-stable 2020-09-18 v20.8.2.3-stable 2020-09-08 v20.7.3.7-stable 2020-09-18 v20.7.2.30-stable 2020-08-31 +v20.6.7.4-stable 2020-09-18 v20.6.6.7-stable 2020-09-11 v20.6.5.8-stable 2020-09-03 v20.6.4.44-stable 2020-08-20 From 122e34cf19081952c3df8f37e3d639df1fdfdaa3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Sep 2020 17:38:28 +0300 Subject: [PATCH 63/68] Fix "Arcadia" --- src/Common/ya.make | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/ya.make b/src/Common/ya.make index 4d558b4dbc9..72f1fa42756 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -87,7 +87,6 @@ SRCS( StatusFile.cpp StatusInfo.cpp Stopwatch.cpp - StringRef.cpp StringUtils/StringUtils.cpp StudentTTest.cpp SymbolIndex.cpp From eae9950abe4e58e03c4d70c1d062dfde8a5417b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Sep 2020 17:56:42 +0300 Subject: [PATCH 64/68] s3uploader: adjust help --- utils/s3tools/s3uploader | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/s3tools/s3uploader b/utils/s3tools/s3uploader index 91fb60ed962..4e8722e0851 100755 --- a/utils/s3tools/s3uploader +++ b/utils/s3tools/s3uploader @@ -68,8 +68,8 @@ def make_tar_file_for_table(clickhouse_data_path, db_name, table_name, USAGE_EXAMPLES = ''' examples: -\ts3uploader --dataset-name some_ds --access-key-id XXX --secret-access-key YYY --clickhouse-data-path /opt/clickhouse/ --table-name default.some_tbl --bucket-name some-bucket -\ts3uploader --dataset-name some_ds --access-key-id XXX --secret-access-key YYY --file-name some_ds.tsv.xz --bucket-name some-bucket +\t./s3uploader --dataset-name some_ds --access-key-id XXX --secret-access-key YYY --clickhouse-data-path /opt/clickhouse/ --table-name default.some_tbl --bucket-name some-bucket +\t./s3uploader --dataset-name some_ds --access-key-id XXX --secret-access-key YYY --file-path some_ds.tsv.xz --bucket-name some-bucket --s3-path /path/to/ ''' if __name__ == "__main__": From 76483b8ed3b90525567620ce4d2f8620d762bf1f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 18 Sep 2020 19:14:19 +0300 Subject: [PATCH 65/68] Update version_date.tsv after release 20.3.19.4 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 9c5f062eade..3ec9ee11b95 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -19,6 +19,7 @@ v20.4.5.36-stable 2020-06-10 v20.4.4.18-stable 2020-05-26 v20.4.3.16-stable 2020-05-23 v20.4.2.9-stable 2020-05-12 +v20.3.19.4-lts 2020-09-18 v20.3.18.10-lts 2020-09-08 v20.3.17.173-lts 2020-08-15 v20.3.16.165-lts 2020-08-08 From 8a8e4f8a41e59e4ce73b823cdf721af02dce92d6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 13:28:54 +0300 Subject: [PATCH 66/68] Attempt to speed-up clang build --- src/Common/PODArray.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index 1084f0800cc..7bd9550500e 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -214,6 +214,9 @@ public: void clear() { c_end = c_start; } template +#if defined(__clang__) + ALWAYS_INLINE /// Better performance in clang build, worse performance in gcc build. +#endif void reserve(size_t n, TAllocatorParams &&... allocator_params) { if (n > capacity()) From 1f429b3d5298e146b51ca77305b3a43786ae0e52 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 17 Sep 2020 13:39:13 +0300 Subject: [PATCH 67/68] Set enable tests to zero for performance build --- docker/packager/packager | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/packager/packager b/docker/packager/packager index 909f20acd6d..0a14102ec04 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -105,6 +105,7 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ # Create combined output archive for split build and for performance tests. if package_type == "performance": result.append("COMBINED_OUTPUT=performance") + cmake_flags.append("-DENABLE_TESTS=0") elif split_binary: result.append("COMBINED_OUTPUT=shared_build") From 368ddada33d19b547bf7bcdbc22a2d057aedb8a5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 17 Sep 2020 17:01:48 +0300 Subject: [PATCH 68/68] Fix binary build --- docker/packager/binary/build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 72adba5d762..dc34e7297dc 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -20,7 +20,7 @@ rm -f CMakeCache.txt cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=$BUILD_TYPE -DSANITIZE=$SANITIZER $CMAKE_FLAGS .. ninja $NINJA_FLAGS clickhouse-bundle mv ./programs/clickhouse* /output -mv ./src/unit_tests_dbms /output +mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds find . -name '*.so' -print -exec mv '{}' /output \; find . -name '*.so.*' -print -exec mv '{}' /output \;