From e44e1ad0d4cb7b08e4b1de3cf863f060e4d493c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jun 2020 21:57:38 +0300 Subject: [PATCH 001/273] 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 002/273] 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 003/273] Fix clang-static-analyzer --- .../MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index b1d9eeac728..b78282dad16 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -219,7 +219,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( String block_id_path = deduplicate_block ? storage.zookeeper_path + "/blocks/" + block_id : ""; auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); - Int64 block_number; + Int64 block_number = 0; String existing_part_name; if (block_number_lock) { @@ -259,6 +259,8 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( part->name = existing_part_name; part->info = MergeTreePartInfo::fromPartName(existing_part_name, storage.format_version); + block_number = part->info.min_block; + /// Don't do subsequent duplicate check. block_id_path.clear(); } From 91e78672fa3fb3b74c233516ae85a7ecd89b7c14 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jun 2020 15:19:29 +0300 Subject: [PATCH 004/273] Enable compact parts by default for small parts --- src/Interpreters/SystemLog.cpp | 3 +-- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index b432cd8803b..c2e82646de9 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -56,8 +56,7 @@ std::shared_ptr createSystemLog( else { String partition_by = config.getString(config_prefix + ".partition_by", "toYYYYMM(event_date)"); - engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time)" - "SETTINGS min_bytes_for_wide_part = '10M'"; /// Use polymorphic parts for log tables by default + engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time)"; } size_t flush_interval_milliseconds = config.getUInt64(config_prefix + ".flush_interval_milliseconds", DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index f2d2a7cc3d4..d88fe26454c 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -29,7 +29,7 @@ struct MergeTreeSettings : public SettingsCollection M(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \ \ /** Data storing format settings. */ \ - M(SettingUInt64, min_bytes_for_wide_part, 0, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ + M(SettingUInt64, min_bytes_for_wide_part, 10485760, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ M(SettingUInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \ \ /** Merge settings. */ \ From c5f46b37e6a044c3a690678c7542f749a265a546 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jun 2020 00:18:27 +0300 Subject: [PATCH 005/273] Update some tests but not all --- ...system_columns_and_system_tables.reference | 2 +- .../00804_test_delta_codec_compression.sql | 12 ++++----- ...ndex_granularity_collapsing_merge_tree.sql | 18 +++++++------ ..._adaptive_index_granularity_merge_tree.sql | 25 +++++++++++-------- ...index_granularity_replacing_merge_tree.sql | 10 ++++---- ...larity_versioned_collapsing_merge_tree.sql | 6 ++--- ...test_fix_extra_seek_on_compressed_cache.sh | 2 +- .../queries/0_stateless/00933_ttl_simple.sql | 10 +------- ...hecksums_in_system_parts_columns_table.sql | 12 ++++----- .../0_stateless/01039_mergetree_exec_time.sql | 2 +- ...1042_check_query_and_last_granule_size.sql | 4 +-- .../01045_order_by_pk_special_storages.sh | 12 ++++----- .../0_stateless/01055_compact_parts.sql | 3 ++- .../00152_insert_different_granularity.sql | 4 +-- 14 files changed, 61 insertions(+), 61 deletions(-) diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference b/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference index 1d1177ba447..ff02b0ba702 100644 --- a/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference +++ b/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference @@ -6,7 +6,7 @@ │ name2 │ 1 │ 0 │ 0 │ 0 │ │ name3 │ 0 │ 0 │ 0 │ 0 │ └───────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘ -231 1 +147 1 ┌─name────────────────┬─partition_key─┬─sorting_key───┬─primary_key─┬─sampling_key─┐ │ check_system_tables │ date │ date, version │ date │ │ └─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘ diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql index ad104eff92c..91bc45df63d 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql @@ -7,12 +7,12 @@ DROP TABLE IF EXISTS default_codec_synthetic; CREATE TABLE delta_codec_synthetic ( id UInt64 Codec(Delta, ZSTD(3)) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; CREATE TABLE default_codec_synthetic ( id UInt64 Codec(ZSTD(3)) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO delta_codec_synthetic SELECT number FROM system.numbers LIMIT 5000000; INSERT INTO default_codec_synthetic SELECT number FROM system.numbers LIMIT 5000000; @@ -45,12 +45,12 @@ DROP TABLE IF EXISTS default_codec_float; CREATE TABLE delta_codec_float ( id Float64 Codec(Delta, LZ4HC) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; CREATE TABLE default_codec_float ( id Float64 Codec(LZ4HC) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO delta_codec_float SELECT number FROM numbers(1547510400, 500000) WHERE number % 3 == 0 OR number % 5 == 0 OR number % 7 == 0 OR number % 11 == 0; INSERT INTO default_codec_float SELECT * from delta_codec_float; @@ -83,12 +83,12 @@ DROP TABLE IF EXISTS default_codec_string; CREATE TABLE delta_codec_string ( id Float64 Codec(Delta, LZ4) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; CREATE TABLE default_codec_string ( id Float64 Codec(LZ4) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO delta_codec_string SELECT concat(toString(number), toString(number % 100)) FROM numbers(1547510400, 500000); INSERT INTO default_codec_string SELECT * from delta_codec_string; diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql index b610d605e23..5603b722513 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql @@ -8,10 +8,11 @@ CREATE TABLE zero_rows_per_granule ( v2 Int64, Sign Int8 ) ENGINE CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(p) ORDER BY k - SETTINGS index_granularity_bytes=20, write_final_mark = 0, - enable_vertical_merge_algorithm=1, - vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + SETTINGS index_granularity_bytes=20, write_final_mark = 0, + min_bytes_for_wide_part = 0, + enable_vertical_merge_algorithm=1, + vertical_merge_algorithm_min_rows_to_activate=0, + vertical_merge_algorithm_min_columns_to_activate=0; INSERT INTO zero_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, 1), ('2018-05-16', 2, 3000, 4000, 1), ('2018-05-17', 3, 5000, 6000, 1), ('2018-05-18', 4, 7000, 8000, 1); @@ -39,10 +40,11 @@ CREATE TABLE four_rows_per_granule ( v2 Int64, Sign Int8 ) ENGINE CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(p) ORDER BY k - SETTINGS index_granularity_bytes=110, write_final_mark = 0, - enable_vertical_merge_algorithm=1, - vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + SETTINGS index_granularity_bytes=110, write_final_mark = 0, + min_bytes_for_wide_part = 0, + enable_vertical_merge_algorithm=1, + vertical_merge_algorithm_min_rows_to_activate=0, + vertical_merge_algorithm_min_columns_to_activate=0; INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, 1), ('2018-05-16', 2, 3000, 4000, 1), ('2018-05-17', 3, 5000, 6000, 1), ('2018-05-18', 4, 7000, 8000, 1); diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.sql index 18ebebb316c..48b6fae19fe 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.sql @@ -7,7 +7,7 @@ CREATE TABLE zero_rows_per_granule ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, write_final_mark = 0; +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO zero_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -34,7 +34,7 @@ CREATE TABLE two_rows_per_granule ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 40, write_final_mark = 0; +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 40, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO two_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -61,7 +61,7 @@ CREATE TABLE four_rows_per_granule ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0; +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO four_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -95,7 +95,7 @@ CREATE TABLE huge_granularity_small_blocks ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 1000000, write_final_mark = 0; +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 1000000, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO huge_granularity_small_blocks (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -126,7 +126,7 @@ CREATE TABLE adaptive_granularity_alter ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0; +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO adaptive_granularity_alter (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -183,7 +183,8 @@ CREATE TABLE zero_rows_per_granule ( SETTINGS index_granularity_bytes=20, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0; INSERT INTO zero_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -215,7 +216,8 @@ CREATE TABLE two_rows_per_granule ( SETTINGS index_granularity_bytes=40, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0; INSERT INTO two_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -246,7 +248,8 @@ CREATE TABLE four_rows_per_granule ( SETTINGS index_granularity_bytes = 110, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0; INSERT INTO four_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -284,7 +287,8 @@ CREATE TABLE huge_granularity_small_blocks ( SETTINGS index_granularity_bytes=1000000, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0; INSERT INTO huge_granularity_small_blocks (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -319,7 +323,8 @@ CREATE TABLE adaptive_granularity_alter ( SETTINGS index_granularity_bytes=110, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0; INSERT INTO adaptive_granularity_alter (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_replacing_merge_tree.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_replacing_merge_tree.sql index f72d5f0f9cb..53a546f9d0f 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_replacing_merge_tree.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_replacing_merge_tree.sql @@ -10,7 +10,7 @@ CREATE TABLE zero_rows_per_granule ( SETTINGS index_granularity_bytes=20, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO zero_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -41,7 +41,7 @@ CREATE TABLE two_rows_per_granule ( SETTINGS index_granularity_bytes=40, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO two_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -72,7 +72,7 @@ CREATE TABLE four_rows_per_granule ( SETTINGS index_granularity_bytes = 110, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO four_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -118,7 +118,7 @@ CREATE TABLE huge_granularity_small_blocks ( SETTINGS index_granularity_bytes=1000000, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO huge_granularity_small_blocks (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -154,7 +154,7 @@ CREATE TABLE adaptive_granularity_alter ( SETTINGS index_granularity_bytes=110, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO adaptive_granularity_alter (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.sql index c5b65839b2a..05f4dc835e5 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.sql @@ -12,7 +12,7 @@ CREATE TABLE zero_rows_per_granule ( SETTINGS index_granularity_bytes=20, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO zero_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 2, 3000, 4000, 1, 1), ('2018-05-17', 3, 5000, 6000, 1, 1), ('2018-05-18', 4, 7000, 8000, 1, 1); @@ -44,7 +44,7 @@ CREATE TABLE four_rows_per_granule ( SETTINGS index_granularity_bytes=120, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 2, 3000, 4000, 1, 1), ('2018-05-17', 3, 5000, 6000, 1, 1), ('2018-05-18', 4, 7000, 8000, 1, 1); @@ -89,7 +89,7 @@ CREATE TABLE six_rows_per_granule ( SETTINGS index_granularity_bytes=170, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO six_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 1, 1000, 2000, -1, 2); diff --git a/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh b/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh index 1f7571a2404..e0225f0d31d 100755 --- a/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh +++ b/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS small_table" -$CLICKHOUSE_CLIENT --query="CREATE TABLE small_table (a UInt64 default 0, n UInt64) ENGINE = MergeTree() PARTITION BY tuple() ORDER BY (a);" +$CLICKHOUSE_CLIENT --query="CREATE TABLE small_table (a UInt64 default 0, n UInt64) ENGINE = MergeTree() PARTITION BY tuple() ORDER BY (a) SETTINGS min_bytes_for_wide_part = 0;" $CLICKHOUSE_CLIENT --query="INSERT INTO small_table(n) SELECT * from system.numbers limit 100000;" diff --git a/tests/queries/0_stateless/00933_ttl_simple.sql b/tests/queries/0_stateless/00933_ttl_simple.sql index b924faad3f5..83d9962043d 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.sql +++ b/tests/queries/0_stateless/00933_ttl_simple.sql @@ -13,9 +13,8 @@ create table ttl_00933_1 (d DateTime, a Int, b Int) engine = MergeTree order by insert into ttl_00933_1 values (now(), 1, 2); insert into ttl_00933_1 values (now(), 3, 4); insert into ttl_00933_1 values (now() + 1000, 5, 6); +select sleep(1.1) format Null; optimize table ttl_00933_1 final; -- check ttl merge for part with both expired and unexpired values -select sleep(1.1) format Null; -- wait if very fast merge happen -optimize table ttl_00933_1 final; select a, b from ttl_00933_1; drop table if exists ttl_00933_1; @@ -24,7 +23,6 @@ create table ttl_00933_1 (d DateTime, a Int ttl d + interval 1 DAY) engine = Mer insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 3); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1 order by d; @@ -34,7 +32,6 @@ create table ttl_00933_1 (d DateTime, a Int) engine = MergeTree order by tuple() insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl_00933_1 values (toDateTime('2100-10-10 00:00:00'), 3); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1 order by d; @@ -43,7 +40,6 @@ drop table if exists ttl_00933_1; create table ttl_00933_1 (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) ttl d + interval 1 day; insert into ttl_00933_1 values (toDate('2000-10-10'), 1); insert into ttl_00933_1 values (toDate('2100-10-10'), 2); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1 order by d; @@ -52,7 +48,6 @@ drop table if exists ttl_00933_1; create table ttl_00933_1 (b Int, a Int ttl now()-1000) engine = MergeTree order by tuple() partition by tuple(); show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1; @@ -61,7 +56,6 @@ drop table if exists ttl_00933_1; create table ttl_00933_1 (b Int, a Int ttl now()+1000) engine = MergeTree order by tuple() partition by tuple(); show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1; @@ -70,7 +64,6 @@ drop table if exists ttl_00933_1; create table ttl_00933_1 (b Int, a Int ttl today()-1) engine = MergeTree order by tuple() partition by tuple(); show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1; @@ -79,7 +72,6 @@ drop table if exists ttl_00933_1; create table ttl_00933_1 (b Int, a Int ttl today()+1) engine = MergeTree order by tuple() partition by tuple(); show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1; diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql index b9eed1e8183..e865ed609be 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql @@ -1,14 +1,14 @@ DROP TABLE IF EXISTS test_00961; -CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree(d, (a, b), 111); +CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree(d, (a, b), 111) SETTINGS min_bytes_for_wide_part = 0; INSERT INTO test_00961 VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789); -SELECT - name, - table, - hash_of_all_files, - hash_of_uncompressed_files, +SELECT + name, + table, + hash_of_all_files, + hash_of_uncompressed_files, uncompressed_hash_of_compressed_files FROM system.parts WHERE table = 'test_00961' and database = currentDatabase(); diff --git a/tests/queries/0_stateless/01039_mergetree_exec_time.sql b/tests/queries/0_stateless/01039_mergetree_exec_time.sql index 4cefb2e9305..d3aade41cea 100644 --- a/tests/queries/0_stateless/01039_mergetree_exec_time.sql +++ b/tests/queries/0_stateless/01039_mergetree_exec_time.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS tab; -create table tab (A Int64) Engine=MergeTree order by tuple(); +create table tab (A Int64) Engine=MergeTree order by tuple() SETTINGS min_bytes_for_wide_part = 0; insert into tab select cityHash64(number) from numbers(1000); select sum(sleep(0.1)) from tab settings max_block_size = 1, max_execution_time=1; -- { serverError 159 } DROP TABLE IF EXISTS tab; diff --git a/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql b/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql index 9777ea1dc45..c62fe25a041 100644 --- a/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql +++ b/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql @@ -1,7 +1,7 @@ SET check_query_single_value_result = 0; DROP TABLE IF EXISTS check_query_test; -CREATE TABLE check_query_test (SomeKey UInt64, SomeValue String) ENGINE = MergeTree() ORDER BY SomeKey; +CREATE TABLE check_query_test (SomeKey UInt64, SomeValue String) ENGINE = MergeTree() ORDER BY SomeKey SETTINGS min_bytes_for_wide_part = 0; -- Number of rows in last granule should be equals to granularity. -- Rows in this table are short, so granularity will be 8192. @@ -17,7 +17,7 @@ DROP TABLE IF EXISTS check_query_test; DROP TABLE IF EXISTS check_query_test_non_adaptive; -CREATE TABLE check_query_test_non_adaptive (SomeKey UInt64, SomeValue String) ENGINE = MergeTree() ORDER BY SomeKey SETTINGS index_granularity_bytes = 0; +CREATE TABLE check_query_test_non_adaptive (SomeKey UInt64, SomeValue String) ENGINE = MergeTree() ORDER BY SomeKey SETTINGS index_granularity_bytes = 0, min_bytes_for_wide_part = 0; INSERT INTO check_query_test_non_adaptive SELECT number, toString(number) FROM system.numbers LIMIT 81920; diff --git a/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh b/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh index 0898fec802c..b0d63b75dd5 100755 --- a/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh +++ b/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh @@ -10,11 +10,11 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS s2" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS m" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS buf" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mv" - -$CLICKHOUSE_CLIENT -q "CREATE TABLE s1 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3" -$CLICKHOUSE_CLIENT -q "CREATE TABLE s2 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3" -$CLICKHOUSE_CLIENT -q "CREATE TABLE m (a UInt32, s String) engine = Merge(currentDatabase(), 's[1,2]')" +$CLICKHOUSE_CLIENT -q "CREATE TABLE s1 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3 SETTINGS min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE s2 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3 SETTINGS min_bytes_for_wide_part = 0" + +$CLICKHOUSE_CLIENT -q "CREATE TABLE m (a UInt32, s String) engine = Merge(currentDatabase(), 's[1,2]') SETTINGS min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "INSERT INTO s1 select (number % 20) * 2 as n, toString(number * number) from numbers(100000)" $CLICKHOUSE_CLIENT -q "INSERT INTO s2 select (number % 20) * 2 + 1 as n, toString(number * number * number) from numbers(100000)" @@ -45,7 +45,7 @@ else fi $CLICKHOUSE_CLIENT -q "SELECT '---MaterializedView---'" -$CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW mv (a UInt32, s String) engine = MergeTree ORDER BY s POPULATE AS SELECT a, s FROM s1 WHERE a % 7 = 0" +$CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW mv (a UInt32, s String) engine = MergeTree ORDER BY s SETTINGS min_bytes_for_wide_part = 0 POPULATE AS SELECT a, s FROM s1 WHERE a % 7 = 0" $CLICKHOUSE_CLIENT -q "SELECT a, s FROM mv ORDER BY s LIMIT 10" rows_read=`$CLICKHOUSE_CLIENT -q "SELECT a, s FROM mv ORDER BY s LIMIT 10 FORMAT JSON" --max_threads=1 --max_block_size=20 | grep "rows_read" | sed 's/[^0-9]*//g'` @@ -59,4 +59,4 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS s1" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS s2" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS m" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS buf" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mv" \ No newline at end of file +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mv" diff --git a/tests/queries/0_stateless/01055_compact_parts.sql b/tests/queries/0_stateless/01055_compact_parts.sql index 05b0657ba7e..e99af76439c 100755 --- a/tests/queries/0_stateless/01055_compact_parts.sql +++ b/tests/queries/0_stateless/01055_compact_parts.sql @@ -3,9 +3,10 @@ set mutations_sync = 2; drop table if exists mt_compact; create table mt_compact(a UInt64, b UInt64 DEFAULT a * a, s String, n Nested(x UInt32, y String), lc LowCardinality(String)) -engine = MergeTree +engine = MergeTree order by a partition by a % 10 settings index_granularity = 8, +min_bytes_for_wide_part = 0, min_rows_for_wide_part = 10; insert into mt_compact (a, s, n.y, lc) select number, toString((number * 2132214234 + 5434543) % 2133443), ['a', 'b', 'c'], number % 2 ? 'bar' : 'baz' from numbers(90); diff --git a/tests/queries/1_stateful/00152_insert_different_granularity.sql b/tests/queries/1_stateful/00152_insert_different_granularity.sql index 5ca34bbe48e..7e04aedf2dd 100644 --- a/tests/queries/1_stateful/00152_insert_different_granularity.sql +++ b/tests/queries/1_stateful/00152_insert_different_granularity.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS fixed_granularity_table; -CREATE TABLE fixed_granularity_table (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, index_granularity_bytes=0; -- looks like default table before update +CREATE TABLE fixed_granularity_table (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, index_granularity_bytes=0, min_bytes_for_wide_part = 0; -- looks like default table before update ALTER TABLE fixed_granularity_table REPLACE PARTITION 201403 FROM test.hits; @@ -29,7 +29,7 @@ ALTER TABLE test.hits ATTACH PARTITION 201403; DROP TABLE IF EXISTS hits_copy; -CREATE TABLE hits_copy (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, index_granularity_bytes=0; +CREATE TABLE hits_copy (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, index_granularity_bytes=0, min_bytes_for_wide_part = 0; ALTER TABLE hits_copy REPLACE PARTITION 201403 FROM test.hits; From 2d43519e038ded3bef0962a3d5ff7a5da7248914 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Jul 2020 02:27:20 +0300 Subject: [PATCH 006/273] Update some tests --- .../0_stateless/00160_merge_and_index_in_in.sql | 4 ++-- .../0_stateless/00443_preferred_block_size_bytes.sh | 6 +++--- ...0484_preferred_max_column_in_block_size_bytes.sql | 8 ++++---- .../00653_verification_monotonic_data_load.sh | 12 ++++++------ ...checksums_in_system_parts_columns_table.reference | 2 +- ...00961_checksums_in_system_parts_columns_table.sql | 2 +- .../01045_order_by_pk_special_storages.sh | 2 +- .../0_stateless/01343_min_bytes_to_use_mmap_io.sql | 2 +- .../01344_min_bytes_to_use_mmap_io_index.sql | 2 +- 9 files changed, 20 insertions(+), 20 deletions(-) diff --git a/tests/queries/0_stateless/00160_merge_and_index_in_in.sql b/tests/queries/0_stateless/00160_merge_and_index_in_in.sql index 6e2838afe88..bdab3f7640d 100644 --- a/tests/queries/0_stateless/00160_merge_and_index_in_in.sql +++ b/tests/queries/0_stateless/00160_merge_and_index_in_in.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS mt_00160; DROP TABLE IF EXISTS merge_00160; -CREATE TABLE mt_00160 (d Date DEFAULT toDate('2015-05-01'), x UInt64) ENGINE = MergeTree(d, x, 1); +CREATE TABLE mt_00160 (d Date DEFAULT toDate('2015-05-01'), x UInt64) ENGINE = MergeTree PARTITION BY d ORDER BY x SETTINGS index_granularity = 1, min_bytes_for_wide_part = 0; CREATE TABLE merge_00160 (d Date, x UInt64) ENGINE = Merge(currentDatabase(), '^mt_00160$'); SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0; @@ -14,7 +14,7 @@ SELECT *, b FROM merge_00160 WHERE x IN (12345, 67890) AND NOT ignore(blockSize( DROP TABLE merge_00160; DROP TABLE mt_00160; -CREATE TABLE mt_00160 (d Date DEFAULT toDate('2015-05-01'), x UInt64, y UInt64, z UInt64) ENGINE = MergeTree(d, (x, z), 1); +CREATE TABLE mt_00160 (d Date DEFAULT toDate('2015-05-01'), x UInt64, y UInt64, z UInt64) ENGINE = MergeTree PARTITION BY d ORDER BY (x, z) SETTINGS index_granularity = 1, min_bytes_for_wide_part = 0; INSERT INTO mt_00160 (x, y, z) SELECT number AS x, number + 10 AS y, number / 2 AS z FROM system.numbers LIMIT 100000; diff --git a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh index 4bf104a2d03..c05611783bb 100755 --- a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh +++ b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS preferred_block_size_bytes" -$CLICKHOUSE_CLIENT -q "CREATE TABLE preferred_block_size_bytes (p Date, s String) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=1, index_granularity_bytes=0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE preferred_block_size_bytes (p Date, s String) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=1, index_granularity_bytes=0, min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "INSERT INTO preferred_block_size_bytes (s) SELECT '16_bytes_-_-_-_' AS s FROM system.numbers LIMIT 10, 90" $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE preferred_block_size_bytes" $CLICKHOUSE_CLIENT --preferred_block_size_bytes=26 -q "SELECT DISTINCT blockSize(), ignore(p, s) FROM preferred_block_size_bytes" @@ -17,7 +17,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS preferred_block_size_bytes" # PREWHERE using empty column $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS pbs" -$CLICKHOUSE_CLIENT -q "CREATE TABLE pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=100, index_granularity_bytes=0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=100, index_granularity_bytes=0, min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "INSERT INTO pbs (p, i, sa) SELECT toDate(i % 30) AS p, number AS i, ['a'] AS sa FROM system.numbers LIMIT 1000" $CLICKHOUSE_CLIENT -q "ALTER TABLE pbs ADD COLUMN s UInt8 DEFAULT 0" $CLICKHOUSE_CLIENT --preferred_block_size_bytes=100000 -q "SELECT count() FROM pbs PREWHERE s = 0" @@ -28,7 +28,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE pbs" # Nullable PREWHERE $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS nullable_prewhere" -$CLICKHOUSE_CLIENT -q "CREATE TABLE nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=8, index_granularity_bytes=0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=8, index_granularity_bytes=0, min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "INSERT INTO nullable_prewhere SELECT toDate(0) AS p, if(number % 2 = 0, CAST(number AS Nullable(UInt64)), CAST(NULL AS Nullable(UInt64))) AS f, number as d FROM system.numbers LIMIT 1001" $CLICKHOUSE_CLIENT -q "SELECT sum(d), sum(f), max(d) FROM nullable_prewhere PREWHERE NOT isNull(f)" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS nullable_prewhere" diff --git a/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql b/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql index e3ab4180d4e..dc021ad52db 100644 --- a/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql +++ b/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql @@ -1,5 +1,5 @@ drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree(date, (date, x), 8192); +create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; insert into tab_00484 select today(), number, toFixedString('', 128) from system.numbers limit 8192; set preferred_block_size_bytes = 2000000; @@ -15,19 +15,19 @@ set preferred_max_column_in_block_size_bytes = 4194304; select max(blockSize()), min(blockSize()), any(ignore(*)) from tab_00484; drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree(date, (date, x), 32); +create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; insert into tab_00484 select today(), number, toFixedString('', 128) from system.numbers limit 47; set preferred_max_column_in_block_size_bytes = 1152; select blockSize(), * from tab_00484 where x = 1 or x > 36 format Null; drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree(date, (date, x), 8192); +create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; insert into tab_00484 select today(), number, toFixedString('', 128) from system.numbers limit 10; set preferred_max_column_in_block_size_bytes = 128; select s from tab_00484 where s == '' format Null; drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s String) engine = MergeTree(date, (date, x), 8192); +create table tab_00484 (date Date, x UInt64, s String) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; insert into tab_00484 select today(), number, 'abc' from system.numbers limit 81920; set preferred_block_size_bytes = 0; select count(*) from tab_00484 prewhere s != 'abc' format Null; diff --git a/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh b/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh index e52610f03ba..3a8c2445e24 100755 --- a/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh +++ b/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh @@ -20,12 +20,12 @@ ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS unsigned_integer_test_table;" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS enum_test_table;" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS date_test_table;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE string_test_table (val String) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE fixed_string_test_table (val FixedString(1)) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE signed_integer_test_table (val Int32) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE unsigned_integer_test_table (val UInt32) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE enum_test_table (val Enum16('hello' = 1, 'world' = 2, 'yandex' = 256, 'clickhouse' = 257)) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE date_test_table (val Date) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE string_test_table (val String) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE fixed_string_test_table (val FixedString(1)) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE signed_integer_test_table (val Int32) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE unsigned_integer_test_table (val UInt32) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE enum_test_table (val Enum16('hello' = 1, 'world' = 2, 'yandex' = 256, 'clickhouse' = 257)) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE date_test_table (val Date) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1, index_granularity_bytes = 0, min_bytes_for_wide_part = 0;" ${CLICKHOUSE_CLIENT} --query="SYSTEM STOP MERGES;" diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference index 282b0ddca7b..3bcfc00eded 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference @@ -1 +1 @@ -20000101_20000101_1_1_0 test_00961 1c63ae7a38eb76e2a71c28aaf0b3ae4d 0053df9b467cc5483e752ec62e91cfd4 da96ff1e527a8a1f908ddf2b1d0af239 +20000101_20000101_1_1_0 test_00961 b78f351b7498ecc9d4732ad29c3952de 1d4b7fbf05d0fc5c2f4559ca75aa32f7 38f047b57fd1bb81cf77e273deb34218 diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql index e865ed609be..f3a729dd4fd 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS test_00961; -CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree(d, (a, b), 111) SETTINGS min_bytes_for_wide_part = 0; +CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree(d, (a, b), 111); INSERT INTO test_00961 VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789); diff --git a/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh b/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh index b0d63b75dd5..3c549fa64ff 100755 --- a/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh +++ b/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh @@ -14,7 +14,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mv" $CLICKHOUSE_CLIENT -q "CREATE TABLE s1 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3 SETTINGS min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "CREATE TABLE s2 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3 SETTINGS min_bytes_for_wide_part = 0" -$CLICKHOUSE_CLIENT -q "CREATE TABLE m (a UInt32, s String) engine = Merge(currentDatabase(), 's[1,2]') SETTINGS min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE m (a UInt32, s String) engine = Merge(currentDatabase(), 's[1,2]')" $CLICKHOUSE_CLIENT -q "INSERT INTO s1 select (number % 20) * 2 as n, toString(number * number) from numbers(100000)" $CLICKHOUSE_CLIENT -q "INSERT INTO s2 select (number % 20) * 2 + 1 as n, toString(number * number * number) from numbers(100000)" diff --git a/tests/queries/0_stateless/01343_min_bytes_to_use_mmap_io.sql b/tests/queries/0_stateless/01343_min_bytes_to_use_mmap_io.sql index 9ff16ca60a7..62c5d20d714 100644 --- a/tests/queries/0_stateless/01343_min_bytes_to_use_mmap_io.sql +++ b/tests/queries/0_stateless/01343_min_bytes_to_use_mmap_io.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS test_01343; -CREATE TABLE test_01343 (x String) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE test_01343 (x String) ENGINE = MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO test_01343 VALUES ('Hello, world'); SET min_bytes_to_use_mmap_io = 1; diff --git a/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql b/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql index 67baef7136d..544c0af7925 100644 --- a/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql +++ b/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS test_01344; -CREATE TABLE test_01344 (x String, INDEX idx (x) TYPE set(10) GRANULARITY 1) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE test_01344 (x String, INDEX idx (x) TYPE set(10) GRANULARITY 1) ENGINE = MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO test_01344 VALUES ('Hello, world'); SET min_bytes_to_use_mmap_io = 1; From 05a5a13e08f615986815e66b75a231824239f558 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 10 Aug 2020 19:23:58 +0800 Subject: [PATCH 007/273] specific ReplicatedMergeTree settings --- .../server-configuration-parameters/settings.md | 16 ++++++++++++++++ programs/server/Server.cpp | 1 + src/Interpreters/Context.cpp | 17 +++++++++++++++++ src/Interpreters/Context.h | 1 + src/Server/ReplicasStatusHandler.cpp | 2 +- .../MergeTree/registerStorageMergeTree.cpp | 6 +++++- 6 files changed, 41 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index c1ac1d0d92d..ee0373c70b4 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -521,6 +521,22 @@ For more information, see the MergeTreeSettings.h header file. ``` +## replicated\_merge\_tree {#server_configuration_parameters-replicated_merge_tree} + +Fine tuning for tables in the [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/mergetree.md). + +This setting has higher priority. + +For more information, see the MergeTreeSettings.h header file. + +**Example** + +``` xml + + 5 + +``` + ## openSSL {#server_configuration_parameters-openssl} SSL client/server configuration. diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 3a975325851..37228fd4a0e 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -596,6 +596,7 @@ int Server::main(const std::vector & /*args*/) /// Check sanity of MergeTreeSettings on server startup global_context->getMergeTreeSettings().sanityCheck(settings); + global_context->getReplicatedMergeTreeSettings().sanityCheck(settings); /// Limit on total memory usage size_t max_server_memory_usage = config().getUInt64("max_server_memory_usage", 0); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 70cf41a679c..431912711f8 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -348,6 +348,7 @@ struct ContextShared mutable std::shared_ptr merge_tree_storage_policy_selector; std::optional merge_tree_settings; /// Settings of MergeTree* engines. + std::optional replicated_merge_tree_settings; /// Settings of ReplicatedMergeTree* engines. std::atomic_size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default) std::atomic_size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default) String format_schema_path; /// Path to a directory that contains schema files used by input formats. @@ -1823,6 +1824,22 @@ const MergeTreeSettings & Context::getMergeTreeSettings() const return *shared->merge_tree_settings; } +const MergeTreeSettings & Context::getReplicatedMergeTreeSettings() const +{ + auto lock = getLock(); + + if (!shared->replicated_merge_tree_settings) + { + const auto & config = getConfigRef(); + MergeTreeSettings mt_settings; + mt_settings.loadFromConfig("merge_tree", config); + mt_settings.loadFromConfig("replicated_merge_tree", config); + shared->replicated_merge_tree_settings.emplace(mt_settings); + } + + return *shared->replicated_merge_tree_settings; +} + const StorageS3Settings & Context::getStorageS3Settings() const { #if !defined(ARCADIA_BUILD) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index c8d13baa9ae..609440e5602 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -537,6 +537,7 @@ public: std::shared_ptr getPartLog(const String & part_database); const MergeTreeSettings & getMergeTreeSettings() const; + const MergeTreeSettings & getReplicatedMergeTreeSettings() const; const StorageS3Settings & getStorageS3Settings() const; /// Prevents DROP TABLE if its size is greater than max_size (50GB by default, max_size=0 turn off this check) diff --git a/src/Server/ReplicasStatusHandler.cpp b/src/Server/ReplicasStatusHandler.cpp index de68635d26e..bc5436f00ee 100644 --- a/src/Server/ReplicasStatusHandler.cpp +++ b/src/Server/ReplicasStatusHandler.cpp @@ -33,7 +33,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request /// Even if lag is small, output detailed information about the lag. bool verbose = params.get("verbose", "") == "1"; - const MergeTreeSettings & settings = context.getMergeTreeSettings(); + const MergeTreeSettings & settings = context.getReplicatedMergeTreeSettings(); bool ok = true; std::stringstream message; diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 4526b0d4f9b..6ee63dd251f 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -514,7 +514,11 @@ static StoragePtr create(const StorageFactory::Arguments & args) StorageInMemoryMetadata metadata; metadata.columns = args.columns; - std::unique_ptr storage_settings = std::make_unique(args.context.getMergeTreeSettings()); + std::unique_ptr storage_settings; + if (replicated) + storage_settings = std::make_unique(args.context.getReplicatedMergeTreeSettings()); + else + storage_settings = std::make_unique(args.context.getMergeTreeSettings()); if (is_extended_storage_def) { From 44364a5f59fa5d55adeea210dab9186281cd4a09 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 11 Aug 2020 21:38:25 +0800 Subject: [PATCH 008/273] add tests --- .../__init__.py | 0 .../configs/config.xml | 9 +++++ .../test_replicated_merge_tree_config/test.py | 37 +++++++++++++++++++ 3 files changed, 46 insertions(+) create mode 100644 tests/integration/test_replicated_merge_tree_config/__init__.py create mode 100644 tests/integration/test_replicated_merge_tree_config/configs/config.xml create mode 100644 tests/integration/test_replicated_merge_tree_config/test.py diff --git a/tests/integration/test_replicated_merge_tree_config/__init__.py b/tests/integration/test_replicated_merge_tree_config/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_replicated_merge_tree_config/configs/config.xml b/tests/integration/test_replicated_merge_tree_config/configs/config.xml new file mode 100644 index 00000000000..d760d05f1bc --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_config/configs/config.xml @@ -0,0 +1,9 @@ + + + + 100 + + + 200 + + diff --git a/tests/integration/test_replicated_merge_tree_config/test.py b/tests/integration/test_replicated_merge_tree_config/test.py new file mode 100644 index 00000000000..bcf6517782d --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_config/test.py @@ -0,0 +1,37 @@ +import pytest +from helpers.cluster import ClickHouseCluster + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node", config_dir="configs", with_zookeeper=True, + ) + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def drop_table(cluster): + yield + for node in cluster.instances.values(): + node.query("DROP TABLE IF EXISTS test1") + node.query("DROP TABLE IF EXISTS test2") + + +def test_replicated_merge_tree_settings(cluster): + node = cluster.instances["node"] + node.query("CREATE TABLE test1 (id Int64) ENGINE MergeTree ORDER BY id") + node.query( + "CREATE TABLE test2 (id Int64) ENGINE ReplicatedMergeTree('/clickhouse/test', 'test') ORDER BY id" + ) + + assert node.query("SHOW CREATE test1").endswith("100") + assert node.query("SHOW CREATE test2").endswith("200") From 1476a9e23642a31f32ae9dd66d87ff005e821fbd Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 23 Aug 2020 20:38:52 +0800 Subject: [PATCH 009/273] Add replicated_merge_tree_settings table --- .../System/StorageSystemMergeTreeSettings.cpp | 11 ++++++++--- .../System/StorageSystemMergeTreeSettings.h | 14 +++++++++----- src/Storages/System/attachSystemTables.cpp | 3 ++- 3 files changed, 19 insertions(+), 9 deletions(-) diff --git a/src/Storages/System/StorageSystemMergeTreeSettings.cpp b/src/Storages/System/StorageSystemMergeTreeSettings.cpp index 4de600ac036..19cbf76f252 100644 --- a/src/Storages/System/StorageSystemMergeTreeSettings.cpp +++ b/src/Storages/System/StorageSystemMergeTreeSettings.cpp @@ -7,7 +7,8 @@ namespace DB { -NamesAndTypesList SystemMergeTreeSettings::getNamesAndTypes() +template +NamesAndTypesList SystemMergeTreeSettings::getNamesAndTypes() { return { {"name", std::make_shared()}, @@ -18,9 +19,11 @@ NamesAndTypesList SystemMergeTreeSettings::getNamesAndTypes() }; } -void SystemMergeTreeSettings::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const +template +void SystemMergeTreeSettings::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - for (const auto & setting : context.getMergeTreeSettings().all()) + const auto & settings = replicated ? context.getReplicatedMergeTreeSettings().all() : context.getMergeTreeSettings().all(); + for (const auto & setting : settings) { res_columns[0]->insert(setting.getName()); res_columns[1]->insert(setting.getValueString()); @@ -30,4 +33,6 @@ void SystemMergeTreeSettings::fillData(MutableColumns & res_columns, const Conte } } +template class SystemMergeTreeSettings; +template class SystemMergeTreeSettings; } diff --git a/src/Storages/System/StorageSystemMergeTreeSettings.h b/src/Storages/System/StorageSystemMergeTreeSettings.h index ac4d9d27505..9f61fa6f780 100644 --- a/src/Storages/System/StorageSystemMergeTreeSettings.h +++ b/src/Storages/System/StorageSystemMergeTreeSettings.h @@ -11,18 +11,22 @@ namespace DB class Context; -/** implements system table "merge_tree_settings", which allows to get information about the current MergeTree settings. +/** implements system table "merge_tree_settings" and "replicated_merge_tree_settings", + * which allows to get information about the current MergeTree settings. */ -class SystemMergeTreeSettings final : public ext::shared_ptr_helper, public IStorageSystemOneBlock +template +class SystemMergeTreeSettings final : public ext::shared_ptr_helper>, + public IStorageSystemOneBlock> { - friend struct ext::shared_ptr_helper; + friend struct ext::shared_ptr_helper>; + public: - std::string getName() const override { return "SystemMergeTreeSettings"; } + std::string getName() const override { return replicated ? "SystemReplicatedMergeTreeSettings" : "SystemMergeTreeSettings"; } static NamesAndTypesList getNamesAndTypes(); protected: - using IStorageSystemOneBlock::IStorageSystemOneBlock; + using IStorageSystemOneBlock>::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 36e4e34361b..2b7ee363f05 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -82,7 +82,8 @@ void attachSystemTablesLocal(IDatabase & system_database) attach(system_database, "functions"); attach(system_database, "events"); attach(system_database, "settings"); - attach(system_database, "merge_tree_settings"); + attach>(system_database, "merge_tree_settings"); + attach>(system_database, "replicated_merge_tree_settings"); attach(system_database, "build_options"); attach(system_database, "formats"); attach(system_database, "table_functions"); From 88db4938f5ec53d343d76789db2d084cb84b5e1f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Aug 2020 02:22:00 +0300 Subject: [PATCH 010/273] 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 011/273] 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 012/273] 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 013/273] 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 014/273] 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 015/273] 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 016/273] 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 017/273] 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 018/273] 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 33a65063cee4a8f1314f7f5c064cbaefcbd59269 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sun, 30 Aug 2020 22:53:22 +0300 Subject: [PATCH 019/273] Docs for the output_format_pretty_max_value_width setting (English). --- docs/en/operations/settings/settings.md | 50 +++++++++++++++++++++++++ 1 file changed, 50 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 4995c04f712..791e3023686 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1899,4 +1899,54 @@ Possible values: Default value: `120`. +## output_format_pretty_max_value_width {#output_format_pretty_max_value_width} + +Limits the width of value displayed in [Pretty](../../interfaces/formats.md#pretty) formats. If the value width exceeds the setting, the value is cut. + +Possible values: + +- Positive integer. Type: [UInt64](../../sql-reference/data-types/int-uint.md). +- 0 — The value is cut completely. + +Default value: `10000` symbols. + +**Examples** + +Query: +```sql +SET output_format_pretty_max_value_width = 10; +SELECT range(number) FROM system.numbers LIMIT 10 FORMAT PrettyCompactNoEscapes; +``` +Result: +```text +┌─range(number)─┐ +│ [] │ +│ [0] │ +│ [0,1] │ +│ [0,1,2] │ +│ [0,1,2,3] │ +│ [0,1,2,3,4⋯ │ +│ [0,1,2,3,4⋯ │ +│ [0,1,2,3,4⋯ │ +│ [0,1,2,3,4⋯ │ +│ [0,1,2,3,4⋯ │ +└───────────────┘ +``` + +Query with zero width: +```sql +SET output_format_pretty_max_value_width = 0; +SELECT range(number) FROM system.numbers LIMIT 5 FORMAT PrettyCompactNoEscapes; +``` +Result: +```text +┌─range(number)─┐ +│ ⋯ │ +│ ⋯ │ +│ ⋯ │ +│ ⋯ │ +│ ⋯ │ +└───────────────┘ +``` + [Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) From 045e4f8964b1939d14251240c592839f33fa7c7b Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sun, 30 Aug 2020 23:03:16 +0300 Subject: [PATCH 020/273] Minor fix. --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 791e3023686..11ab1247753 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1905,7 +1905,7 @@ Limits the width of value displayed in [Pretty](../../interfaces/formats.md#pret Possible values: -- Positive integer. Type: [UInt64](../../sql-reference/data-types/int-uint.md). +- Positive integer. - 0 — The value is cut completely. Default value: `10000` symbols. From 0db5b4a72ceb27a39b79c4b975f6c119e9057e29 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 2 Sep 2020 02:43:23 +0300 Subject: [PATCH 021/273] 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 022/273] 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 023/273] 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 024/273] 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 23fb122818d54e18225ea71562b35d4c82b005ac Mon Sep 17 00:00:00 2001 From: Gao Qiang <30835199+dreamerfable@users.noreply.github.com> Date: Wed, 2 Sep 2020 23:27:27 +0800 Subject: [PATCH 025/273] Update replacingmergetree.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit fix the wrong translation of sorting key fix the wrong format make some sentences more understandable I think the title which is a name of table engine is more approriate to keeping in english 。 --- .../mergetree-family/replacingmergetree.md | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/docs/zh/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/zh/engines/table-engines/mergetree-family/replacingmergetree.md index 626597eeaf0..73328015ea9 100644 --- a/docs/zh/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/zh/engines/table-engines/mergetree-family/replacingmergetree.md @@ -1,8 +1,8 @@ -# 替换合并树 {#replacingmergetree} +# ReplacingMergeTree {#replacingmergetree} -该引擎和[MergeTree](mergetree.md)的不同之处在于它会删除具有相同主键的重复项。 +该引擎和 [MergeTree](mergetree.md) 的不同之处在于它会删除排序键值相同的重复项。 -数据的去重只会在合并的过程中出现。合并会在未知的时间在后台进行,因此你无法预先作出计划。有一些数据可能仍未被处理。尽管你可以调用 `OPTIMIZE` 语句发起计划外的合并,但请不要指望使用它,因为 `OPTIMIZE` 语句会引发对大量数据的读和写。 +数据的去重只会在数据合并期间进行。合并会在后台一个不确定的时间进行,因此你无法预先作出计划。有一些数据可能仍未被处理。尽管你可以调用 `OPTIMIZE` 语句发起计划外的合并,但请不要依靠它,因为 `OPTIMIZE` 语句会引发对数据的大量读写。 因此,`ReplacingMergeTree` 适用于在后台清除重复的数据以节省空间,但是它不保证没有重复的数据出现。 @@ -21,19 +21,20 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [SETTINGS name=value, ...] ``` -请求参数的描述,参考[请求参数](../../../engines/table-engines/mergetree-family/replacingmergetree.md)。 +有关建表参数的描述,可参考 [创建表](../../../sql-reference/statements/create.md#create-table-query)。 -**参数** +**ReplacingMergeTree 的参数** - `ver` — 版本列。类型为 `UInt*`, `Date` 或 `DateTime`。可选参数。 - 合并的时候,`ReplacingMergeTree` 从所有具有相同主键的行中选择一行留下: - - 如果 `ver` 列未指定,选择最后一条。 - - 如果 `ver` 列已指定,选择 `ver` 值最大的版本。 + 在数据合并的时候,`ReplacingMergeTree` 从所有具有相同排序键的行中选择一行留下: + + - 如果 `ver` 列未指定,保留最后一条。 + - 如果 `ver` 列已指定,保留 `ver` 值最大的版本。 **子句** -创建 `ReplacingMergeTree` 表时,需要与创建 `MergeTree` 表时相同的[子句](mergetree.md)。 +创建 `ReplacingMergeTree` 表时,需要使用与创建 `MergeTree` 表时相同的 [子句](mergetree.md)。
From a321d6970c2ff77a3dd6a650a5ac696e2cff3e80 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 3 Sep 2020 04:42:25 +0300 Subject: [PATCH 026/273] 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 dd0feeeaa139ddce8e171adbcaf9824cfffb66b2 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 3 Sep 2020 11:03:28 +0800 Subject: [PATCH 027/273] ISSUES-4006 support datetime with precision --- src/Core/MySQL/MySQLReplication.cpp | 34 +++++++++++++++---- src/Core/MySQL/MySQLReplication.h | 32 +++++++++++++---- .../MySQL/MaterializeMySQLSyncThread.cpp | 3 ++ 3 files changed, 56 insertions(+), 13 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 41afe3cde6a..1b5ca132eeb 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -451,7 +451,7 @@ namespace MySQLReplication UInt32 hour = readBits(val, 2, 10, 24); UInt32 minute = readBits(val, 12, 6, 24); UInt32 second = readBits(val, 18, 6, 24); - readTimeFractionalPart(payload, reinterpret_cast(&frac_part), meta); + readTimeFractionalPart(payload, frac_part, meta); if (frac_part != 0) { @@ -481,9 +481,10 @@ namespace MySQLReplication break; } case MYSQL_TYPE_DATETIME2: { - Int64 val = 0, fsp = 0; + Int64 val = 0; + UInt32 fsp = 0; readBigEndianStrict(payload, reinterpret_cast(&val), 5); - readTimeFractionalPart(payload, reinterpret_cast(&fsp), meta); + readTimeFractionalPart(payload, fsp, meta); UInt32 year_month = readBits(val, 1, 17, 40); time_t date_time = DateLUT::instance().makeDateTime( @@ -491,14 +492,35 @@ namespace MySQLReplication , readBits(val, 23, 5, 40), readBits(val, 28, 6, 40), readBits(val, 34, 6, 40) ); - row.push_back(Field{UInt32(date_time)}); + if (!meta) + row.push_back(Field{UInt32(date_time)}); + else + { + DB::DecimalUtils::DecimalComponents components{ + static_cast(date_time), 0}; + + components.fractional = fsp; + row.push_back(Field(DecimalUtils::decimalFromComponents(components, meta))); + } + break; } case MYSQL_TYPE_TIMESTAMP2: { UInt32 sec = 0, fsp = 0; readBigEndianStrict(payload, reinterpret_cast(&sec), 4); - readTimeFractionalPart(payload, reinterpret_cast(&fsp), meta); - row.push_back(Field{sec}); + readTimeFractionalPart(payload, fsp, meta); + + if (!meta) + row.push_back(Field{sec}); + else + { + DB::DecimalUtils::DecimalComponents components{ + static_cast(sec), 0}; + + components.fractional = fsp; + row.push_back(Field(DecimalUtils::decimalFromComponents(components, meta))); + } + break; } case MYSQL_TYPE_NEWDECIMAL: { diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index 7834a1b355c..aac075fae2f 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -37,23 +37,41 @@ namespace MySQLReplication std::reverse(start, end); } - inline void readTimeFractionalPart(ReadBuffer & payload, char * to, UInt16 meta) + inline void readTimeFractionalPart(ReadBuffer & payload, UInt32 & factional, UInt16 meta) { switch (meta) { case 1: - case 2: { - readBigEndianStrict(payload, to, 1); + { + readBigEndianStrict(payload, reinterpret_cast(&factional), 1); + factional /= 10; + break; + } + case 2: + { + readBigEndianStrict(payload, reinterpret_cast(&factional), 1); break; } case 3: - case 4: { - readBigEndianStrict(payload, to, 2); + { + readBigEndianStrict(payload, reinterpret_cast(&factional), 2); + factional /= 10; + break; + } + case 4: + { + readBigEndianStrict(payload, reinterpret_cast(&factional), 2); break; } case 5: - case 6: { - readBigEndianStrict(payload, to, 3); + { + readBigEndianStrict(payload, reinterpret_cast(&factional), 3); + factional /= 10; + break; + } + case 6: + { + readBigEndianStrict(payload, reinterpret_cast(&factional), 3); break; } default: diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 851ea351876..7ded256dd86 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -9,6 +9,7 @@ # include # include # include +# include # include # include # include @@ -451,6 +452,8 @@ static void writeFieldsToColumn( write_data_to_column(casted_float32_column, Float64(), Float32()); else if (ColumnFloat64 * casted_float64_column = typeid_cast(&column_to)) write_data_to_column(casted_float64_column, Float64(), Float64()); + else if (ColumnDecimal * casted_date_time_64_column = typeid_cast *>(&column_to)) + write_data_to_column(casted_date_time_64_column, DateTime64(), DateTime64()); else if (ColumnInt32 * casted_int32_column = typeid_cast(&column_to)) { for (size_t index = 0; index < rows_data.size(); ++index) From 9c091fb2c1e251f4714ea4415fe33ce49a767e70 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 3 Sep 2020 11:08:38 +0800 Subject: [PATCH 028/273] ISSUES-4006 remove unsupport data type --- src/Core/MySQL/MySQLReplication.cpp | 126 ++-------------------------- 1 file changed, 5 insertions(+), 121 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 1b5ca132eeb..557b84dd9bf 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -198,10 +198,9 @@ namespace MySQLReplication case MYSQL_TYPE_LONGLONG: case MYSQL_TYPE_INT24: case MYSQL_TYPE_DATE: - case MYSQL_TYPE_TIME: case MYSQL_TYPE_DATETIME: - case MYSQL_TYPE_YEAR: - case MYSQL_TYPE_NEWDATE: { + case MYSQL_TYPE_NEWDATE: + { /// No data here. column_meta.emplace_back(0); break; @@ -211,24 +210,21 @@ namespace MySQLReplication case MYSQL_TYPE_DOUBLE: case MYSQL_TYPE_TIMESTAMP2: case MYSQL_TYPE_DATETIME2: - case MYSQL_TYPE_TIME2: - case MYSQL_TYPE_JSON: case MYSQL_TYPE_BLOB: - case MYSQL_TYPE_GEOMETRY: { + { column_meta.emplace_back(UInt16(meta[pos])); pos += 1; break; } case MYSQL_TYPE_NEWDECIMAL: - case MYSQL_TYPE_STRING: { + case MYSQL_TYPE_STRING: + { auto b0 = UInt16(meta[pos] << 8); auto b1 = UInt8(meta[pos + 1]); column_meta.emplace_back(UInt16(b0 + b1)); pos += 2; break; } - - case MYSQL_TYPE_BIT: case MYSQL_TYPE_VARCHAR: case MYSQL_TYPE_VAR_STRING: { auto b0 = UInt8(meta[pos]); @@ -405,21 +401,6 @@ namespace MySQLReplication row.push_back(Field{val}); break; } - case MYSQL_TYPE_TIME: { - UInt32 i24 = 0; - payload.readStrict(reinterpret_cast(&i24), 3); - - String time_buff; - time_buff.resize(8); - sprintf( - time_buff.data(), - "%02d:%02d:%02d", - static_cast(i24 / 10000), - static_cast(i24 % 10000) / 100, - static_cast(i24 % 100)); - row.push_back(Field{String{time_buff}}); - break; - } case MYSQL_TYPE_DATE: { UInt32 i24 = 0; payload.readStrict(reinterpret_cast(&i24), 3); @@ -430,56 +411,6 @@ namespace MySQLReplication row.push_back(Field(date_day_number.toUnderType())); break; } - case MYSQL_TYPE_YEAR: { - Int32 val = 0; - payload.readStrict(reinterpret_cast(&val), 1); - - String time_buff; - time_buff.resize(4); - sprintf(time_buff.data(), "%04d", (val + 1900)); - row.push_back(Field{String{time_buff}}); - break; - } - case MYSQL_TYPE_TIME2: { - UInt32 val = 0, frac_part = 0; - - readBigEndianStrict(payload, reinterpret_cast(&val), 3); - if (readBits(val, 0, 1, 24) == 0) - { - val = ~val + 1; - } - UInt32 hour = readBits(val, 2, 10, 24); - UInt32 minute = readBits(val, 12, 6, 24); - UInt32 second = readBits(val, 18, 6, 24); - readTimeFractionalPart(payload, frac_part, meta); - - if (frac_part != 0) - { - String time_buff; - time_buff.resize(15); - sprintf( - time_buff.data(), - "%02d:%02d:%02d.%06d", - static_cast(hour), - static_cast(minute), - static_cast(second), - static_cast(frac_part)); - row.push_back(Field{String{time_buff}}); - } - else - { - String time_buff; - time_buff.resize(8); - sprintf( - time_buff.data(), - "%02d:%02d:%02d", - static_cast(hour), - static_cast(minute), - static_cast(second)); - row.push_back(Field{String{time_buff}}); - } - break; - } case MYSQL_TYPE_DATETIME2: { Int64 val = 0; UInt32 fsp = 0; @@ -607,42 +538,6 @@ namespace MySQLReplication row.push_back(Field{String{format}}); break; } - case MYSQL_TYPE_ENUM: { - Int32 val = 0; - Int32 len = (meta & 0xff); - switch (len) - { - case 1: { - payload.readStrict(reinterpret_cast(&val), 1); - break; - } - case 2: { - payload.readStrict(reinterpret_cast(&val), 2); - break; - } - default: - break; - } - row.push_back(Field{Int32{val}}); - break; - } - case MYSQL_TYPE_BIT: { - UInt32 bits = ((meta >> 8) * 8) + (meta & 0xff); - UInt32 size = (bits + 7) / 8; - - Bitmap bitmap1; - readBitmap(payload, bitmap1, size); - row.push_back(Field{UInt64{bitmap1.to_ulong()}}); - break; - } - case MYSQL_TYPE_SET: { - UInt32 size = (meta & 0xff); - - Bitmap bitmap1; - readBitmap(payload, bitmap1, size); - row.push_back(Field{UInt64{bitmap1.to_ulong()}}); - break; - } case MYSQL_TYPE_VARCHAR: case MYSQL_TYPE_VAR_STRING: { uint32_t size = 0; @@ -678,7 +573,6 @@ namespace MySQLReplication row.push_back(Field{String{val}}); break; } - case MYSQL_TYPE_GEOMETRY: case MYSQL_TYPE_BLOB: { UInt32 size = 0; switch (meta) @@ -709,16 +603,6 @@ namespace MySQLReplication row.push_back(Field{String{val}}); break; } - case MYSQL_TYPE_JSON: { - UInt32 size = 0; - payload.readStrict(reinterpret_cast(&size), meta); - - String val; - val.resize(size); - payload.readStrict(reinterpret_cast(val.data()), size); - row.push_back(Field{String{val}}); - break; - } default: throw ReplicationError( "ParseRow: Unhandled MySQL field type:" + std::to_string(field_type), ErrorCodes::UNKNOWN_EXCEPTION); From 802a5a31a0e6a9328cf5868f094c100927ca52c4 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 3 Sep 2020 11:11:18 +0800 Subject: [PATCH 029/273] ISSUES-4006 fix code style --- src/Core/MySQL/MySQLReplication.cpp | 45 +++++++++++++++++++---------- 1 file changed, 30 insertions(+), 15 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 557b84dd9bf..6a102dca1f3 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -352,56 +352,65 @@ namespace MySQLReplication switch (field_type) { - case MYSQL_TYPE_TINY: { + case MYSQL_TYPE_TINY: + { UInt8 val = 0; payload.readStrict(reinterpret_cast(&val), 1); row.push_back(Field{UInt8{val}}); break; } - case MYSQL_TYPE_SHORT: { + case MYSQL_TYPE_SHORT: + { UInt16 val = 0; payload.readStrict(reinterpret_cast(&val), 2); row.push_back(Field{UInt16{val}}); break; } - case MYSQL_TYPE_INT24: { + case MYSQL_TYPE_INT24: + { Int32 val = 0; payload.readStrict(reinterpret_cast(&val), 3); row.push_back(Field{Int32{val}}); break; } - case MYSQL_TYPE_LONG: { + case MYSQL_TYPE_LONG: + { UInt32 val = 0; payload.readStrict(reinterpret_cast(&val), 4); row.push_back(Field{UInt32{val}}); break; } - case MYSQL_TYPE_LONGLONG: { + case MYSQL_TYPE_LONGLONG: + { UInt64 val = 0; payload.readStrict(reinterpret_cast(&val), 8); row.push_back(Field{UInt64{val}}); break; } - case MYSQL_TYPE_FLOAT: { + case MYSQL_TYPE_FLOAT: + { Float32 val = 0; payload.readStrict(reinterpret_cast(&val), 4); row.push_back(Field{Float32{val}}); break; } - case MYSQL_TYPE_DOUBLE: { + case MYSQL_TYPE_DOUBLE: + { Float64 val = 0; payload.readStrict(reinterpret_cast(&val), 8); row.push_back(Field{Float64{val}}); break; } - case MYSQL_TYPE_TIMESTAMP: { + case MYSQL_TYPE_TIMESTAMP: + { UInt32 val = 0; payload.readStrict(reinterpret_cast(&val), 4); row.push_back(Field{val}); break; } - case MYSQL_TYPE_DATE: { + case MYSQL_TYPE_DATE: + { UInt32 i24 = 0; payload.readStrict(reinterpret_cast(&i24), 3); @@ -411,7 +420,8 @@ namespace MySQLReplication row.push_back(Field(date_day_number.toUnderType())); break; } - case MYSQL_TYPE_DATETIME2: { + case MYSQL_TYPE_DATETIME2: + { Int64 val = 0; UInt32 fsp = 0; readBigEndianStrict(payload, reinterpret_cast(&val), 5); @@ -436,7 +446,8 @@ namespace MySQLReplication break; } - case MYSQL_TYPE_TIMESTAMP2: { + case MYSQL_TYPE_TIMESTAMP2: + { UInt32 sec = 0, fsp = 0; readBigEndianStrict(payload, reinterpret_cast(&sec), 4); readTimeFractionalPart(payload, fsp, meta); @@ -454,7 +465,8 @@ namespace MySQLReplication break; } - case MYSQL_TYPE_NEWDECIMAL: { + case MYSQL_TYPE_NEWDECIMAL: + { Int8 digits_per_integer = 9; Int8 precision = meta >> 8; Int8 decimals = meta & 0xff; @@ -539,7 +551,8 @@ namespace MySQLReplication break; } case MYSQL_TYPE_VARCHAR: - case MYSQL_TYPE_VAR_STRING: { + case MYSQL_TYPE_VAR_STRING: + { uint32_t size = 0; if (meta < 256) { @@ -556,7 +569,8 @@ namespace MySQLReplication row.push_back(Field{String{val}}); break; } - case MYSQL_TYPE_STRING: { + case MYSQL_TYPE_STRING: + { UInt32 size = 0; if (field_len < 256) { @@ -573,7 +587,8 @@ namespace MySQLReplication row.push_back(Field{String{val}}); break; } - case MYSQL_TYPE_BLOB: { + case MYSQL_TYPE_BLOB: + { UInt32 size = 0; switch (meta) { From 9fef663caad63f65232b4951a3451ceb55c2af76 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 3 Sep 2020 21:06:30 +0300 Subject: [PATCH 030/273] add test --- .../01457_order_by_nulls_first.reference | 0 .../01457_order_by_nulls_first.sql | 26 +++++++++++++++++++ 2 files changed, 26 insertions(+) create mode 100644 tests/queries/0_stateless/01457_order_by_nulls_first.reference create mode 100644 tests/queries/0_stateless/01457_order_by_nulls_first.sql diff --git a/tests/queries/0_stateless/01457_order_by_nulls_first.reference b/tests/queries/0_stateless/01457_order_by_nulls_first.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01457_order_by_nulls_first.sql b/tests/queries/0_stateless/01457_order_by_nulls_first.sql new file mode 100644 index 00000000000..7e391276d84 --- /dev/null +++ b/tests/queries/0_stateless/01457_order_by_nulls_first.sql @@ -0,0 +1,26 @@ +drop table if exists order_by_nulls_first; + +CREATE TABLE order_by_nulls_first +(diff Nullable(Int16), traf UInt64) +ENGINE = MergeTree ORDER BY tuple(); + +insert into order_by_nulls_first values (NULL,1),(NULL,0),(NULL,0),(NULL,0),(NULL,0),(NULL,0),(28,0),(0,0); + +SELECT + diff, + traf +FROM order_by_nulls_first +order by diff desc NULLS FIRST, traf +limit 1, 4; + +select '---'; + +SELECT + diff, + traf +FROM order_by_nulls_first +ORDER BY + diff DESC NULLS FIRST, + traf ASC; + +drop table if exists order_by_nulls_first; \ No newline at end of file From d718c5af9952ce630bae1771dd0eb6839f6ad1f6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 4 Sep 2020 17:36:08 +0300 Subject: [PATCH 031/273] fixed --- src/Columns/ColumnNullable.cpp | 129 +++++++++++------- .../01457_order_by_nulls_first.reference | 76 +++++++++++ .../01457_order_by_nulls_first.sql | 72 +++++++++- 3 files changed, 230 insertions(+), 47 deletions(-) diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 888410202f0..caebe28e510 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -329,73 +329,110 @@ void ColumnNullable::getPermutation(bool reverse, size_t limit, int null_directi } } -void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_direction_hint, IColumn::Permutation & res, EqualRanges & equal_range) const +void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const { - if (limit >= equal_range.back().second || limit >= size()) + if (limit >= equal_ranges.back().second || limit >= size()) limit = 0; - EqualRanges new_ranges, temp_ranges; + EqualRanges new_ranges; - for (const auto &[first, last] : equal_range) + const auto is_nulls_last = ((null_direction_hint > 0) != reverse); + + if (is_nulls_last) { - bool direction = ((null_direction_hint > 0) != reverse); /// Shift all NULL values to the end. - - size_t read_idx = first; - size_t write_idx = first; - while (read_idx < last && (isNullAt(res[read_idx])^direction)) + for (const auto & [first, last] : equal_ranges) { - ++read_idx; - ++write_idx; - } + /// Consider a half interval [first, last) + size_t read_idx = first; + size_t write_idx = first; + size_t end_idx = last; - ++read_idx; + if (!limit) + limit = end_idx; + else + limit = std::min(end_idx - first + 1, limit); - /// Invariants: - /// write_idx < read_idx - /// write_idx points to NULL - /// read_idx will be incremented to position of next not-NULL - /// there are range of NULLs between write_idx and read_idx - 1, - /// We are moving elements from end to begin of this range, - /// so range will "bubble" towards the end. - /// Relative order of NULL elements could be changed, - /// but relative order of non-NULLs is preserved. - - while (read_idx < last && write_idx < last) - { - if (isNullAt(res[read_idx])^direction) + while (read_idx < limit && !isNullAt(res[read_idx])) { - std::swap(res[read_idx], res[write_idx]); + ++read_idx; ++write_idx; } - ++read_idx; - } - if (write_idx - first > 1) - { - if (direction) - temp_ranges.emplace_back(first, write_idx); - else + ++read_idx; + + /// Invariants: + /// write_idx < read_idx + /// write_idx points to NULL + /// read_idx will be incremented to position of next not-NULL + /// there are range of NULLs between write_idx and read_idx - 1, + /// We are moving elements from end to begin of this range, + /// so range will "bubble" towards the end. + /// Relative order of NULL elements could be changed, + /// but relative order of non-NULLs is preserved. + + while (read_idx < end_idx && write_idx < limit) + { + if (!isNullAt(res[read_idx])) + { + std::swap(res[read_idx], res[write_idx]); + ++write_idx; + } + ++read_idx; + } + + /// We have a range [first, write_idx) of non-NULL values + if (first != write_idx) new_ranges.emplace_back(first, write_idx); - } - if (last - write_idx > 1) - { - if (direction) + /// We have a range [write_idx, list) of NULL values + if (write_idx != last) new_ranges.emplace_back(write_idx, last); - else - temp_ranges.emplace_back(write_idx, last); } } - while (!new_ranges.empty() && limit && limit <= new_ranges.back().first) - new_ranges.pop_back(); + else + { + for (const auto & [first, last] : equal_ranges) + { + /// Shift all NULL values to the beginning. - if (!temp_ranges.empty()) - getNestedColumn().updatePermutation(reverse, limit, null_direction_hint, res, temp_ranges); + ssize_t read_idx = last - 1; + ssize_t write_idx = last - 1; + ssize_t begin_idx = first; - equal_range.resize(temp_ranges.size() + new_ranges.size()); - std::merge(temp_ranges.begin(), temp_ranges.end(), new_ranges.begin(), new_ranges.end(), equal_range.begin()); + while (read_idx >= begin_idx && !isNullAt(res[read_idx])) + { + --read_idx; + --write_idx; + } + + --read_idx; + + while (read_idx >= begin_idx && write_idx >= begin_idx) + { + if (!isNullAt(res[read_idx])) + { + std::swap(res[read_idx], res[write_idx]); + --write_idx; + } + --read_idx; + } + + /// We have a range [write_idx+1, last) of non-NULL values + if (write_idx != static_cast(last)) + new_ranges.emplace_back(write_idx + 1, last); + + + /// We have a range [first, write_idx+1) of NULL values + if (static_cast(first) != write_idx) + new_ranges.emplace_back(first, write_idx + 1); + } + } + + getNestedColumn().updatePermutation(reverse, 0, null_direction_hint, res, new_ranges); + + equal_ranges = std::move(new_ranges); } void ColumnNullable::gather(ColumnGathererStream & gatherer) diff --git a/tests/queries/0_stateless/01457_order_by_nulls_first.reference b/tests/queries/0_stateless/01457_order_by_nulls_first.reference index e69de29bb2d..355e58120fe 100644 --- a/tests/queries/0_stateless/01457_order_by_nulls_first.reference +++ b/tests/queries/0_stateless/01457_order_by_nulls_first.reference @@ -0,0 +1,76 @@ +\N 0 +\N 0 +\N 0 +\N 0 +--- DESC NULLS FIRST, ASC +\N 0 +\N 0 +\N 0 +\N 0 +\N 0 +\N 1 +28 0 +0 0 +--- DESC NULLS LAST, ASC +28 0 +0 0 +\N 0 +\N 0 +\N 0 +\N 0 +\N 0 +\N 1 +--- ASC NULLS FIRST, ASC +\N 0 +\N 0 +\N 0 +\N 0 +\N 0 +\N 1 +0 0 +28 0 +--- ASC NULLS LAST, ASC +0 0 +28 0 +\N 0 +\N 0 +\N 0 +\N 0 +\N 0 +\N 1 +--- DESC NULLS FIRST, DESC +\N 1 +\N 0 +\N 0 +\N 0 +\N 0 +\N 0 +28 0 +0 0 +--- DESC NULLS LAST, DESC +28 0 +0 0 +\N 1 +\N 0 +\N 0 +\N 0 +\N 0 +\N 0 +--- ASC NULLS FIRST, DESC +\N 1 +\N 0 +\N 0 +\N 0 +\N 0 +\N 0 +0 0 +28 0 +--- ASC NULLS LAST, DESC +0 0 +28 0 +\N 1 +\N 0 +\N 0 +\N 0 +\N 0 +\N 0 diff --git a/tests/queries/0_stateless/01457_order_by_nulls_first.sql b/tests/queries/0_stateless/01457_order_by_nulls_first.sql index 7e391276d84..100c87fbead 100644 --- a/tests/queries/0_stateless/01457_order_by_nulls_first.sql +++ b/tests/queries/0_stateless/01457_order_by_nulls_first.sql @@ -13,7 +13,7 @@ FROM order_by_nulls_first order by diff desc NULLS FIRST, traf limit 1, 4; -select '---'; +select '--- DESC NULLS FIRST, ASC'; SELECT diff, @@ -23,4 +23,74 @@ ORDER BY diff DESC NULLS FIRST, traf ASC; +select '--- DESC NULLS LAST, ASC'; + +SELECT + diff, + traf +FROM order_by_nulls_first +ORDER BY + diff DESC NULLS LAST, + traf ASC; + +select '--- ASC NULLS FIRST, ASC'; + +SELECT + diff, + traf +FROM order_by_nulls_first +ORDER BY + diff ASC NULLS FIRST, + traf ASC; + +select '--- ASC NULLS LAST, ASC'; + +SELECT + diff, + traf +FROM order_by_nulls_first +ORDER BY + diff ASC NULLS LAST, + traf ASC; + +select '--- DESC NULLS FIRST, DESC'; + +SELECT + diff, + traf +FROM order_by_nulls_first +ORDER BY + diff DESC NULLS FIRST, + traf DESC; + +select '--- DESC NULLS LAST, DESC'; + +SELECT + diff, + traf +FROM order_by_nulls_first +ORDER BY + diff DESC NULLS LAST, + traf DESC; + +select '--- ASC NULLS FIRST, DESC'; + +SELECT + diff, + traf +FROM order_by_nulls_first +ORDER BY + diff ASC NULLS FIRST, + traf DESC; + +select '--- ASC NULLS LAST, DESC'; + +SELECT + diff, + traf +FROM order_by_nulls_first +ORDER BY + diff ASC NULLS LAST, + traf DESC; + drop table if exists order_by_nulls_first; \ No newline at end of file From f67a7b3a3d855c6e0d07a1bda10faaf414270327 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 4 Sep 2020 19:53:50 +0300 Subject: [PATCH 032/273] better --- src/Columns/ColumnNullable.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index caebe28e510..6033cdad53a 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -334,7 +334,8 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire if (limit >= equal_ranges.back().second || limit >= size()) limit = 0; - EqualRanges new_ranges; + /// We will sort nested columns into `new_ranges` and call updatePermutation in next columns with `null_ranges`. + EqualRanges new_ranges, null_ranges; const auto is_nulls_last = ((null_direction_hint > 0) != reverse); @@ -388,7 +389,7 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire /// We have a range [write_idx, list) of NULL values if (write_idx != last) - new_ranges.emplace_back(write_idx, last); + null_ranges.emplace_back(write_idx, last); } } else @@ -426,13 +427,14 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire /// We have a range [first, write_idx+1) of NULL values if (static_cast(first) != write_idx) - new_ranges.emplace_back(first, write_idx + 1); + null_ranges.emplace_back(first, write_idx + 1); } } getNestedColumn().updatePermutation(reverse, 0, null_direction_hint, res, new_ranges); equal_ranges = std::move(new_ranges); + equal_ranges.insert(equal_ranges.end(), null_ranges.begin(), null_ranges.end()); } void ColumnNullable::gather(ColumnGathererStream & gatherer) From 4fadb6c3ecb826807f4e1ec17419e7e6b36b4982 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 4 Sep 2020 21:05:06 +0300 Subject: [PATCH 033/273] fast test --- src/Columns/ColumnArray.cpp | 3 +++ src/Columns/ColumnDecimal.cpp | 3 +++ src/Columns/ColumnFixedString.cpp | 3 +++ src/Columns/ColumnLowCardinality.cpp | 3 +++ src/Columns/ColumnNullable.cpp | 10 +++++++++- src/Columns/ColumnString.cpp | 3 +++ src/Columns/ColumnTuple.cpp | 3 +++ src/Columns/ColumnUnique.h | 3 +++ src/Columns/ColumnVector.cpp | 3 +++ 9 files changed, 33 insertions(+), 1 deletion(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 58543d6a4dd..cd4aa57c18f 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -781,6 +781,9 @@ void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_h void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_range) const { + if (equal_range.empty()) + return; + if (limit >= size() || limit >= equal_range.back().second) limit = 0; diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 6b4341a697e..6bab4228e9d 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -144,6 +144,9 @@ void ColumnDecimal::getPermutation(bool reverse, size_t limit, int , IColumn: template void ColumnDecimal::updatePermutation(bool reverse, size_t limit, int, IColumn::Permutation & res, EqualRanges & equal_range) const { + if (equal_range.empty()) + return; + if (limit >= data.size() || limit >= equal_range.back().second) limit = 0; diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 95a477e54cf..c10caa37b28 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -170,6 +170,9 @@ void ColumnFixedString::getPermutation(bool reverse, size_t limit, int /*nan_dir void ColumnFixedString::updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_range) const { + if (equal_range.empty()) + return; + if (limit >= size() || limit >= equal_range.back().second) limit = 0; diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 5c174d57b32..0613e5e2b71 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -331,6 +331,9 @@ void ColumnLowCardinality::getPermutation(bool reverse, size_t limit, int nan_di void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_range) const { + if (equal_range.empty()) + return; + if (limit >= size() || limit >= equal_range.back().second) limit = 0; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 6033cdad53a..1846e066bed 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -331,6 +331,9 @@ void ColumnNullable::getPermutation(bool reverse, size_t limit, int null_directi void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const { + if (equal_ranges.empty()) + return; + if (limit >= equal_ranges.back().second || limit >= size()) limit = 0; @@ -433,8 +436,13 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire getNestedColumn().updatePermutation(reverse, 0, null_direction_hint, res, new_ranges); + std::cout << "new_ranges " << new_ranges.size() << std::endl; + std::cout << "null_ranges " << null_ranges.size() << std::endl; + equal_ranges = std::move(new_ranges); - equal_ranges.insert(equal_ranges.end(), null_ranges.begin(), null_ranges.end()); + std::move(null_ranges.begin(), null_ranges.end(), std::back_inserter(equal_ranges)); + + std::cout << "end" << std::endl; } void ColumnNullable::gather(ColumnGathererStream & gatherer) diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 6c84107caae..57795535a64 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -327,6 +327,9 @@ void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_directio void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res, EqualRanges & equal_range) const { + if (equal_range.empty()) + return; + if (limit >= size() || limit > equal_range.back().second) limit = 0; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 87e5e37db51..09c7472b22b 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -346,6 +346,9 @@ void ColumnTuple::getPermutation(bool reverse, size_t limit, int nan_direction_h void ColumnTuple::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_range) const { + if (equal_range.empty()) + return; + for (const auto& column : columns) { column->updatePermutation(reverse, limit, nan_direction_hint, res, equal_range); diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index d0edf65edd8..59febe52112 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -389,6 +389,9 @@ int ColumnUnique::compareAt(size_t n, size_t m, const IColumn & rhs, template void ColumnUnique::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_range) const { + if (equal_range.empty()) + return; + bool found_null_value_index = false; for (size_t i = 0; i < equal_range.size() && !found_null_value_index; ++i) { diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index b1b3ed4478a..416123af8f0 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -243,6 +243,9 @@ void ColumnVector::getPermutation(bool reverse, size_t limit, int nan_directi template void ColumnVector::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_range) const { + if (equal_range.empty()) + return; + if (limit >= data.size() || limit >= equal_range.back().second) limit = 0; From f757438e71d8e7a973d778fc0a5f0e54effc8130 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Fri, 4 Sep 2020 22:03:16 +0300 Subject: [PATCH 034/273] Update docs/en/operations/settings/settings.md Co-authored-by: BayoNet --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 11ab1247753..d4edc22a89b 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1901,7 +1901,7 @@ Default value: `120`. ## output_format_pretty_max_value_width {#output_format_pretty_max_value_width} -Limits the width of value displayed in [Pretty](../../interfaces/formats.md#pretty) formats. If the value width exceeds the setting, the value is cut. +Limits the width of value displayed in [Pretty](../../interfaces/formats.md#pretty) formats. If the value width exceeds the limit, the value is cut. Possible values: From e948327b441f3dd499537976509234f82e1bcd19 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Fri, 4 Sep 2020 23:30:25 +0300 Subject: [PATCH 035/273] Translated into Russian. --- docs/ru/operations/settings/settings.md | 54 +++++++++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 62c40c90540..b04f8f411c3 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1643,4 +1643,58 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; - [Секции и настройки запроса CREATE TABLE](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-query-clauses) (настройка `merge_with_ttl_timeout`) - [Table TTL](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-table-ttl) +## output_format_pretty_max_value_width {#output_format_pretty_max_value_width} + +Ограничивает длину значения, выводимого в формате [Pretty](../../interfaces/formats.md#pretty). Если значение длиннее указанного количества символов, оно обрезается. + +Возможные значения: + +- Положительное целое число. +- 0 — значение обрезается полностью. + +Значение по умолчанию: `10000` символов. + +**Примеры** + +Запрос: + +```sql +SET output_format_pretty_max_value_width = 10; +SELECT range(number) FROM system.numbers LIMIT 10 FORMAT PrettyCompactNoEscapes; +``` +Результат: + +```text +┌─range(number)─┐ +│ [] │ +│ [0] │ +│ [0,1] │ +│ [0,1,2] │ +│ [0,1,2,3] │ +│ [0,1,2,3,4⋯ │ +│ [0,1,2,3,4⋯ │ +│ [0,1,2,3,4⋯ │ +│ [0,1,2,3,4⋯ │ +│ [0,1,2,3,4⋯ │ +└───────────────┘ +``` + +Запрос, где длина выводимого значения ограничена 0 символов: + +```sql +SET output_format_pretty_max_value_width = 0; +SELECT range(number) FROM system.numbers LIMIT 5 FORMAT PrettyCompactNoEscapes; +``` +Результат: + +```text +┌─range(number)─┐ +│ ⋯ │ +│ ⋯ │ +│ ⋯ │ +│ ⋯ │ +│ ⋯ │ +└───────────────┘ +``` + [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) From a3671a15f61d4958b5184e97d4048bc454b816c5 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 7 Sep 2020 10:54:59 +0800 Subject: [PATCH 036/273] Add new feature: SHOW DATABASES LIKE --- .../InterpreterShowTablesQuery.cpp | 16 +++++++++++++++- src/Parsers/ASTShowTablesQuery.cpp | 14 ++++++++++++++ src/Parsers/ParserShowTablesQuery.cpp | 19 +++++++++++++++++++ .../01470_show_databases_like.reference | 1 + .../0_stateless/01470_show_databases_like.sql | 3 +++ 5 files changed, 52 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01470_show_databases_like.reference create mode 100644 tests/queries/0_stateless/01470_show_databases_like.sql diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index 4b0d4c21ad1..f8e387ef529 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -31,7 +31,21 @@ String InterpreterShowTablesQuery::getRewrittenQuery() /// SHOW DATABASES if (query.databases) - return "SELECT name FROM system.databases"; + { + std::stringstream rewritten_query; + rewritten_query << "SELECT name FROM system.databases"; + + if (!query.like.empty()) + { + rewritten_query << " WHERE name " << (query.not_like ? "NOT " : "") << "LIKE " << std::quoted(query.like, '\''); + } + + if (query.limit_length) + rewritten_query << " LIMIT " << query.limit_length; + + DUMP(rewritten_query.str()); + return rewritten_query.str(); + } /// SHOW CLUSTER/CLUSTERS if (query.clusters) diff --git a/src/Parsers/ASTShowTablesQuery.cpp b/src/Parsers/ASTShowTablesQuery.cpp index 5a284109cf2..ce44d2b56de 100644 --- a/src/Parsers/ASTShowTablesQuery.cpp +++ b/src/Parsers/ASTShowTablesQuery.cpp @@ -18,6 +18,20 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format if (databases) { settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW DATABASES" << (settings.hilite ? hilite_none : ""); + + if (!like.empty()) + settings.ostr + << (settings.hilite ? hilite_keyword : "") + << (not_like ? " NOT" : "") + << (case_insensitive_like ? " ILIKE " : " LIKE ") + << (settings.hilite ? hilite_none : "") + << std::quoted(like, '\''); + + if (limit_length) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " LIMIT " << (settings.hilite ? hilite_none : ""); + limit_length->formatImpl(settings, state, frame); + } } else if (clusters) { diff --git a/src/Parsers/ParserShowTablesQuery.cpp b/src/Parsers/ParserShowTablesQuery.cpp index 66ecdf61c58..4586e10a8a3 100644 --- a/src/Parsers/ParserShowTablesQuery.cpp +++ b/src/Parsers/ParserShowTablesQuery.cpp @@ -46,6 +46,25 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (s_databases.ignore(pos)) { query->databases = true; + + if (s_not.ignore(pos, expected)) + query->not_like = true; + + if (bool insensitive = s_ilike.ignore(pos, expected); insensitive || s_like.ignore(pos, expected)) + { + if (insensitive) + query->case_insensitive_like = true; + + if (!like_p.parse(pos, like, expected)) + return false; + } + else if (query->not_like) + return false; + if (s_limit.ignore(pos, expected)) + { + if (!exp_elem.parse(pos, query->limit_length, expected)) + return false; + } } else if (s_clusters.ignore(pos)) { diff --git a/tests/queries/0_stateless/01470_show_databases_like.reference b/tests/queries/0_stateless/01470_show_databases_like.reference new file mode 100644 index 00000000000..19b0eb327f5 --- /dev/null +++ b/tests/queries/0_stateless/01470_show_databases_like.reference @@ -0,0 +1 @@ +test_01470 diff --git a/tests/queries/0_stateless/01470_show_databases_like.sql b/tests/queries/0_stateless/01470_show_databases_like.sql new file mode 100644 index 00000000000..46ec8878105 --- /dev/null +++ b/tests/queries/0_stateless/01470_show_databases_like.sql @@ -0,0 +1,3 @@ +create database if not exists test_01470; +show databases like '%01470'; +drop database test_01470; From dc0e276bba24425c9355874e17a28874cfd7e336 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 7 Sep 2020 11:06:21 +0800 Subject: [PATCH 037/273] Add new feature: SHOW DATABASES LIKE --- src/Interpreters/InterpreterShowTablesQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index f8e387ef529..09c617e12ec 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -43,7 +43,6 @@ String InterpreterShowTablesQuery::getRewrittenQuery() if (query.limit_length) rewritten_query << " LIMIT " << query.limit_length; - DUMP(rewritten_query.str()); return rewritten_query.str(); } From b7e9d5e72dcc926d3e9ee3bd426354e93b7d2b5a Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 5 Sep 2020 13:46:51 +0800 Subject: [PATCH 038/273] ISSUES-4006 support decimal for MaterializedMySQL engine --- src/Core/MySQL/IMySQLReadPacket.h | 2 +- src/Core/MySQL/MySQLReplication.cpp | 143 +++++++++--------- .../MySQL/MaterializeMySQLSyncThread.cpp | 10 +- 3 files changed, 80 insertions(+), 75 deletions(-) diff --git a/src/Core/MySQL/IMySQLReadPacket.h b/src/Core/MySQL/IMySQLReadPacket.h index eab31889091..7484e7acc89 100644 --- a/src/Core/MySQL/IMySQLReadPacket.h +++ b/src/Core/MySQL/IMySQLReadPacket.h @@ -25,7 +25,7 @@ protected: virtual void readPayloadImpl(ReadBuffer & buf) = 0; }; -class LimitedReadPacket : public IMySQLReadPacket + class LimitedReadPacket : public IMySQLReadPacket { public: void readPayload(ReadBuffer & in, uint8_t & sequence_id) override; diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 6a102dca1f3..07bc4773882 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -467,87 +467,86 @@ namespace MySQLReplication } case MYSQL_TYPE_NEWDECIMAL: { - Int8 digits_per_integer = 9; - Int8 precision = meta >> 8; - Int8 decimals = meta & 0xff; - const char compressed_byte_map[] = {0, 1, 1, 2, 2, 3, 3, 4, 4, 4}; - - Int8 integral = (precision - decimals); - UInt32 uncompressed_integers = integral / digits_per_integer; - UInt32 uncompressed_decimals = decimals / digits_per_integer; - UInt32 compressed_integers = integral - (uncompressed_integers * digits_per_integer); - UInt32 compressed_decimals = decimals - (uncompressed_decimals * digits_per_integer); - - String buff; - UInt32 bytes_to_read = uncompressed_integers * 4 + compressed_byte_map[compressed_integers] - + uncompressed_decimals * 4 + compressed_byte_map[compressed_decimals]; - buff.resize(bytes_to_read); - payload.readStrict(reinterpret_cast(buff.data()), bytes_to_read); - - String format; - format.resize(0); - - bool is_negative = ((buff[0] & 0x80) == 0); - if (is_negative) + const auto & dispatch = [](const size_t & precision, const size_t & scale, const auto & function) -> Field { - format += "-"; - } - buff[0] ^= 0x80; + if (precision <= DecimalUtils::maxPrecision()) + return Field(function(precision, scale, Decimal32())); + else if (precision <= DecimalUtils::maxPrecision()) + return Field(function(precision, scale, Decimal64())); + else if (precision <= DecimalUtils::maxPrecision()) + return Field(function(precision, scale, Decimal128())); - ReadBufferFromString reader(buff); - /// Compressed part. - if (compressed_integers != 0) - { - Int64 val = 0; - UInt8 to_read = compressed_byte_map[compressed_integers]; - readBigEndianStrict(reader, reinterpret_cast(&val), to_read); - format += std::to_string(val); - } + return Field(function(precision, scale, Decimal256())); + }; - for (auto k = 0U; k < uncompressed_integers; k++) + const auto & read_decimal = [&](const size_t & precision, const size_t & scale, auto decimal) { - UInt32 val = 0; - readBigEndianStrict(reader, reinterpret_cast(&val), 4); - format += std::to_string(val); - } - format += "."; - for (auto k = 0U; k < uncompressed_decimals; k++) - { - UInt32 val = 0; - reader.readStrict(reinterpret_cast(&val), 4); - format += std::to_string(val); - } + using DecimalType = decltype(decimal); + static constexpr size_t digits_per_integer = 9; + static const size_t compressed_byte_map[] = {0, 1, 1, 2, 2, 3, 3, 4, 4, 4}; + + DecimalType res(0); + bool is_negative = (*payload.position() & 0x80) == 0; + *payload.position() ^= 0x80; - /// Compressed part. - if (compressed_decimals != 0) - { - Int64 val = 0; - String compressed_buff; - UInt8 to_read = compressed_byte_map[compressed_decimals]; - switch (to_read) { - case 1: { - reader.readStrict(reinterpret_cast(&val), 1); - break; + size_t integral = (precision - scale); + size_t uncompressed_integers = integral / digits_per_integer; + size_t compressed_integers = integral - (uncompressed_integers * digits_per_integer); + + /// Compressed part. + if (compressed_integers != 0) + { + Int64 val = 0; + size_t to_read = compressed_byte_map[compressed_integers]; + readBigEndianStrict(payload, reinterpret_cast(&val), to_read); + res += val; } - case 2: { - readBigEndianStrict(reader, reinterpret_cast(&val), 2); - break; + + for (auto k = 0U; k < uncompressed_integers; k++) + { + UInt32 val = 0; + readBigEndianStrict(payload, reinterpret_cast(&val), 4); + res *= intExp10OfSize(k ? digits_per_integer : std::max(size_t(1), compressed_integers)); + res += val; } - case 3: { - readBigEndianStrict(reader, reinterpret_cast(&val), 3); - break; - } - case 4: { - readBigEndianStrict(reader, reinterpret_cast(&val), 4); - break; - } - default: - break; } - format += std::to_string(val); - } - row.push_back(Field{String{format}}); + + { + size_t uncompressed_decimals = scale / digits_per_integer; + size_t compressed_decimals = scale - (uncompressed_decimals * digits_per_integer); + + for (auto k = 0U; k < uncompressed_decimals; k++) + { + UInt32 val = 0; + payload.readStrict(reinterpret_cast(&val), 4); + res *= intExp10OfSize(digits_per_integer); + res += val; + } + + /// Compressed part. + if (compressed_decimals != 0) + { + Int64 val = 0; + String compressed_buff; + size_t to_read = compressed_byte_map[compressed_decimals]; + + if (to_read) + { + payload.readStrict(reinterpret_cast(&val), to_read); + res *= intExp10OfSize(compressed_decimals); + res += val; + } + } + } + + if (is_negative) + res *= -1; + + return res; + }; + + row.push_back(dispatch((meta >> 8) & 0xFF, meta & 0xFF, read_decimal)); break; } case MYSQL_TYPE_VARCHAR: diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 7ded256dd86..3a26e25d08f 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -452,8 +452,14 @@ static void writeFieldsToColumn( write_data_to_column(casted_float32_column, Float64(), Float32()); else if (ColumnFloat64 * casted_float64_column = typeid_cast(&column_to)) write_data_to_column(casted_float64_column, Float64(), Float64()); - else if (ColumnDecimal * casted_date_time_64_column = typeid_cast *>(&column_to)) - write_data_to_column(casted_date_time_64_column, DateTime64(), DateTime64()); + else if (ColumnDecimal * casted_decimal_32_column = typeid_cast *>(&column_to)) + write_data_to_column(casted_decimal_32_column, Decimal32(), Decimal32()); + else if (ColumnDecimal * casted_decimal_64_column = typeid_cast *>(&column_to)) + write_data_to_column(casted_decimal_64_column, Decimal64(), Decimal64()); + else if (ColumnDecimal * casted_decimal_128_column = typeid_cast *>(&column_to)) + write_data_to_column(casted_decimal_128_column, Decimal128(), Decimal128()); + else if (ColumnDecimal * casted_decimal_256_column = typeid_cast *>(&column_to)) + write_data_to_column(casted_decimal_256_column, Decimal256(), Decimal256()); else if (ColumnInt32 * casted_int32_column = typeid_cast(&column_to)) { for (size_t index = 0; index < rows_data.size(); ++index) From c09d86e5e460e466bf25ef382cdf0d499112eff4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 7 Sep 2020 16:15:04 +0300 Subject: [PATCH 039/273] another fixes --- src/Columns/ColumnNullable.cpp | 35 ++++++++++++++++++++++++++-------- src/Columns/ColumnVector.cpp | 6 ++++++ src/Interpreters/sortBlock.cpp | 32 +++++++++++++++++++++++++++++++ 3 files changed, 65 insertions(+), 8 deletions(-) diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 1846e066bed..e5771ceed5c 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -331,6 +331,12 @@ void ColumnNullable::getPermutation(bool reverse, size_t limit, int null_directi void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const { +// std::cout << "ColumnNullable" << std::endl; +// +// std::cout << "equal_ranges " << equal_ranges.size() << std::endl; +// for (auto [first, last] : equal_ranges) +// std::cout << "first " << first << " last " << last << std::endl; + if (equal_ranges.empty()) return; @@ -344,20 +350,24 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire if (is_nulls_last) { +// std::cout << "NULL LAST" << std::endl; /// Shift all NULL values to the end. for (const auto & [first, last] : equal_ranges) { +// std::cout << "current range " << first << ' ' << last << std::endl; /// Consider a half interval [first, last) size_t read_idx = first; size_t write_idx = first; size_t end_idx = last; if (!limit) - limit = end_idx; + limit = end_idx - read_idx; else - limit = std::min(end_idx - first + 1, limit); + limit = std::min(end_idx - read_idx, limit); - while (read_idx < limit && !isNullAt(res[read_idx])) + /// We simply check the limit not to do extra work. + /// Since interval begins from `first`, not from zero, we add `first` to the right side of the inequality. + while (read_idx < first + limit && !isNullAt(res[read_idx])) { ++read_idx; ++write_idx; @@ -375,7 +385,7 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire /// Relative order of NULL elements could be changed, /// but relative order of non-NULLs is preserved. - while (read_idx < end_idx && write_idx < limit) + while (read_idx < end_idx && write_idx < first + limit) { if (!isNullAt(res[read_idx])) { @@ -397,6 +407,7 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire } else { +// std::cout << "NULLS FIRST" << std::endl; for (const auto & [first, last] : equal_ranges) { /// Shift all NULL values to the beginning. @@ -436,13 +447,21 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire getNestedColumn().updatePermutation(reverse, 0, null_direction_hint, res, new_ranges); - std::cout << "new_ranges " << new_ranges.size() << std::endl; - std::cout << "null_ranges " << null_ranges.size() << std::endl; - +// std::cout << "new_ranges " << new_ranges.size() << std::endl; +// for (auto [first, last] : new_ranges) +// std::cout << "first " << first << " last " << last << std::endl; +// std::cout << "null_ranges " << null_ranges.size() << std::endl; +// for (auto [first, last] : null_ranges) +// std::cout << "first " << first << " last " << last << std::endl; +// equal_ranges = std::move(new_ranges); std::move(null_ranges.begin(), null_ranges.end(), std::back_inserter(equal_ranges)); - std::cout << "end" << std::endl; +// std::cout << "equal_ranges_final " << equal_ranges.size() << std::endl; +// for (auto [first, last] : equal_ranges) +// std::cout << "first " << first << " last " << last << std::endl; + +// std::cout << "end" << std::endl; } void ColumnNullable::gather(ColumnGathererStream & gatherer) diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 416123af8f0..b42d9409a2a 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -243,6 +243,12 @@ void ColumnVector::getPermutation(bool reverse, size_t limit, int nan_directi template void ColumnVector::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_range) const { +// std::cout << "ColumnVector" << std::endl; +// +// std::cout << "equal_ranges " << equal_range.size() << std::endl; +// for (auto [first, last] : equal_range) +// std::cout << "first " << first << " last " << last << std::endl; + if (equal_range.empty()) return; diff --git a/src/Interpreters/sortBlock.cpp b/src/Interpreters/sortBlock.cpp index cb3c36e5356..1a8b80cbadb 100644 --- a/src/Interpreters/sortBlock.cpp +++ b/src/Interpreters/sortBlock.cpp @@ -104,6 +104,19 @@ struct PartialSortingLessWithCollation void sortBlock(Block & block, const SortDescription & description, UInt64 limit) { +// std::cout << block.dumpStructure() << std::endl; +// +// for (const auto & column : block.getColumnsWithTypeAndName()) +// { +// std::cout << column.name << " \t\t"; +// auto column_size = column.column->size(); +// for (size_t i = 0; i < column_size; ++i) +// { +// std::cout << toString(column.column->operator[](i)) << ", \t"; +// } +// std::cout << std::endl; +// } + if (!block) return; @@ -181,6 +194,8 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) ranges.emplace_back(0, perm.size()); for (const auto & column : columns_with_sort_desc) { +// std::cout << "need collation" << std::endl; +// std::cout << column.column->dumpStructure() << std::endl; while (!ranges.empty() && limit && limit <= ranges.back().first) ranges.pop_back(); @@ -210,6 +225,9 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) ranges.emplace_back(0, perm.size()); for (const auto & column : columns_with_sort_desc) { +// std::cout << "no need collation" << std::endl; +// std::cout << column.column->dumpStructure() << std::endl; + while (!ranges.empty() && limit && limit <= ranges.back().first) { ranges.pop_back(); @@ -229,6 +247,20 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) block.getByPosition(i).column = block.getByPosition(i).column->permute(perm, limit); } } +// +// std::cout << "final block" << std::endl; +// std::cout << block.dumpStructure() << std::endl; +// +// for (const auto & column : block.getColumnsWithTypeAndName()) +// { +// std::cout << column.name << " \t\t"; +// auto column_size = column.column->size(); +// for (size_t i = 0; i < column_size; ++i) +// { +// std::cout << toString(column.column->operator[](i)) << ", \t"; +// } +// std::cout << std::endl; +// } } From 8793281e3e6d6cde788e044af168893d11efe146 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 7 Sep 2020 17:02:59 +0300 Subject: [PATCH 040/273] remove cout --- src/Columns/ColumnNullable.cpp | 22 ---------------------- src/Columns/ColumnVector.cpp | 6 ------ src/Interpreters/sortBlock.cpp | 32 -------------------------------- 3 files changed, 60 deletions(-) diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index e5771ceed5c..12934b9420b 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -331,12 +331,6 @@ void ColumnNullable::getPermutation(bool reverse, size_t limit, int null_directi void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const { -// std::cout << "ColumnNullable" << std::endl; -// -// std::cout << "equal_ranges " << equal_ranges.size() << std::endl; -// for (auto [first, last] : equal_ranges) -// std::cout << "first " << first << " last " << last << std::endl; - if (equal_ranges.empty()) return; @@ -350,11 +344,9 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire if (is_nulls_last) { -// std::cout << "NULL LAST" << std::endl; /// Shift all NULL values to the end. for (const auto & [first, last] : equal_ranges) { -// std::cout << "current range " << first << ' ' << last << std::endl; /// Consider a half interval [first, last) size_t read_idx = first; size_t write_idx = first; @@ -407,7 +399,6 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire } else { -// std::cout << "NULLS FIRST" << std::endl; for (const auto & [first, last] : equal_ranges) { /// Shift all NULL values to the beginning. @@ -447,21 +438,8 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire getNestedColumn().updatePermutation(reverse, 0, null_direction_hint, res, new_ranges); -// std::cout << "new_ranges " << new_ranges.size() << std::endl; -// for (auto [first, last] : new_ranges) -// std::cout << "first " << first << " last " << last << std::endl; -// std::cout << "null_ranges " << null_ranges.size() << std::endl; -// for (auto [first, last] : null_ranges) -// std::cout << "first " << first << " last " << last << std::endl; -// equal_ranges = std::move(new_ranges); std::move(null_ranges.begin(), null_ranges.end(), std::back_inserter(equal_ranges)); - -// std::cout << "equal_ranges_final " << equal_ranges.size() << std::endl; -// for (auto [first, last] : equal_ranges) -// std::cout << "first " << first << " last " << last << std::endl; - -// std::cout << "end" << std::endl; } void ColumnNullable::gather(ColumnGathererStream & gatherer) diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index b42d9409a2a..416123af8f0 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -243,12 +243,6 @@ void ColumnVector::getPermutation(bool reverse, size_t limit, int nan_directi template void ColumnVector::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_range) const { -// std::cout << "ColumnVector" << std::endl; -// -// std::cout << "equal_ranges " << equal_range.size() << std::endl; -// for (auto [first, last] : equal_range) -// std::cout << "first " << first << " last " << last << std::endl; - if (equal_range.empty()) return; diff --git a/src/Interpreters/sortBlock.cpp b/src/Interpreters/sortBlock.cpp index 1a8b80cbadb..cb3c36e5356 100644 --- a/src/Interpreters/sortBlock.cpp +++ b/src/Interpreters/sortBlock.cpp @@ -104,19 +104,6 @@ struct PartialSortingLessWithCollation void sortBlock(Block & block, const SortDescription & description, UInt64 limit) { -// std::cout << block.dumpStructure() << std::endl; -// -// for (const auto & column : block.getColumnsWithTypeAndName()) -// { -// std::cout << column.name << " \t\t"; -// auto column_size = column.column->size(); -// for (size_t i = 0; i < column_size; ++i) -// { -// std::cout << toString(column.column->operator[](i)) << ", \t"; -// } -// std::cout << std::endl; -// } - if (!block) return; @@ -194,8 +181,6 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) ranges.emplace_back(0, perm.size()); for (const auto & column : columns_with_sort_desc) { -// std::cout << "need collation" << std::endl; -// std::cout << column.column->dumpStructure() << std::endl; while (!ranges.empty() && limit && limit <= ranges.back().first) ranges.pop_back(); @@ -225,9 +210,6 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) ranges.emplace_back(0, perm.size()); for (const auto & column : columns_with_sort_desc) { -// std::cout << "no need collation" << std::endl; -// std::cout << column.column->dumpStructure() << std::endl; - while (!ranges.empty() && limit && limit <= ranges.back().first) { ranges.pop_back(); @@ -247,20 +229,6 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) block.getByPosition(i).column = block.getByPosition(i).column->permute(perm, limit); } } -// -// std::cout << "final block" << std::endl; -// std::cout << block.dumpStructure() << std::endl; -// -// for (const auto & column : block.getColumnsWithTypeAndName()) -// { -// std::cout << column.name << " \t\t"; -// auto column_size = column.column->size(); -// for (size_t i = 0; i < column_size; ++i) -// { -// std::cout << toString(column.column->operator[](i)) << ", \t"; -// } -// std::cout << std::endl; -// } } From 46c84b054e8c1a041a5dd58df249f22ca25122d1 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 8 Sep 2020 01:28:20 +0800 Subject: [PATCH 041/273] ISSUES-4006 fix negative decimal number --- src/Core/MySQL/IMySQLReadPacket.h | 2 +- src/Core/MySQL/MySQLReplication.cpp | 31 +++++++++++++++++------------ 2 files changed, 19 insertions(+), 14 deletions(-) diff --git a/src/Core/MySQL/IMySQLReadPacket.h b/src/Core/MySQL/IMySQLReadPacket.h index 7484e7acc89..eab31889091 100644 --- a/src/Core/MySQL/IMySQLReadPacket.h +++ b/src/Core/MySQL/IMySQLReadPacket.h @@ -25,7 +25,7 @@ protected: virtual void readPayloadImpl(ReadBuffer & buf) = 0; }; - class LimitedReadPacket : public IMySQLReadPacket +class LimitedReadPacket : public IMySQLReadPacket { public: void readPayload(ReadBuffer & in, uint8_t & sequence_id) override; diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 07bc4773882..50bbe9aaaf7 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -483,10 +483,16 @@ namespace MySQLReplication { using DecimalType = decltype(decimal); static constexpr size_t digits_per_integer = 9; - static const size_t compressed_byte_map[] = {0, 1, 1, 2, 2, 3, 3, 4, 4, 4}; + static const size_t compressed_bytes_map[] = {0, 1, 1, 2, 2, 3, 3, 4, 4, 4}; + static const size_t compressed_integer_align_numbers[] = { + 0x0, 0xFF, 0xFF, 0xFFFF, 0xFFFF, 0xFFFFFF, 0xFFFFFF, 0xFFFFFFFF, 0xFFFFFFFF, 0xFFFFFFFF}; + UInt32 mask = 0; DecimalType res(0); - bool is_negative = (*payload.position() & 0x80) == 0; + + if ((*payload.position() & 0x80) == 0) + mask = UInt32(-1); + *payload.position() ^= 0x80; { @@ -497,18 +503,18 @@ namespace MySQLReplication /// Compressed part. if (compressed_integers != 0) { - Int64 val = 0; - size_t to_read = compressed_byte_map[compressed_integers]; + UInt32 val = 0; + size_t to_read = compressed_bytes_map[compressed_integers]; readBigEndianStrict(payload, reinterpret_cast(&val), to_read); - res += val; + res += (val ^ (mask & compressed_integer_align_numbers[compressed_integers])); } for (auto k = 0U; k < uncompressed_integers; k++) { UInt32 val = 0; readBigEndianStrict(payload, reinterpret_cast(&val), 4); - res *= intExp10OfSize(k ? digits_per_integer : std::max(size_t(1), compressed_integers)); - res += val; + res *= intExp10OfSize(digits_per_integer); + res += (val ^ mask); } } @@ -521,26 +527,25 @@ namespace MySQLReplication UInt32 val = 0; payload.readStrict(reinterpret_cast(&val), 4); res *= intExp10OfSize(digits_per_integer); - res += val; + res += (val ^ mask); } /// Compressed part. if (compressed_decimals != 0) { - Int64 val = 0; - String compressed_buff; - size_t to_read = compressed_byte_map[compressed_decimals]; + UInt32 val = 0; + size_t to_read = compressed_bytes_map[compressed_decimals]; if (to_read) { payload.readStrict(reinterpret_cast(&val), to_read); res *= intExp10OfSize(compressed_decimals); - res += val; + res += (val ^ (mask & compressed_integer_align_numbers[compressed_decimals])); } } } - if (is_negative) + if (mask != 0) res *= -1; return res; From 564dfac737802b2bb8e39f98eb8914587f7bc97b Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 8 Sep 2020 02:36:07 +0800 Subject: [PATCH 042/273] ISSUES-4006 fix decimal type decimal number part --- src/Core/MySQL/MySQLReplication.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 50bbe9aaaf7..a277c183653 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -525,7 +525,7 @@ namespace MySQLReplication for (auto k = 0U; k < uncompressed_decimals; k++) { UInt32 val = 0; - payload.readStrict(reinterpret_cast(&val), 4); + readBigEndianStrict(payload, reinterpret_cast(&val), 4); res *= intExp10OfSize(digits_per_integer); res += (val ^ mask); } @@ -538,7 +538,7 @@ namespace MySQLReplication if (to_read) { - payload.readStrict(reinterpret_cast(&val), to_read); + readBigEndianStrict(payload, reinterpret_cast(&val), to_read); res *= intExp10OfSize(compressed_decimals); res += (val ^ (mask & compressed_integer_align_numbers[compressed_decimals])); } From 05bd0b7c28672aefbce78297a0e2f354a3a7ac4a Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 8 Sep 2020 02:36:28 +0800 Subject: [PATCH 043/273] ISSUES-4006 add integration test --- .../materialize_with_ddl.py | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 18695f40e53..2bddb7f7c84 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -95,6 +95,26 @@ def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_nam mysql_node.query("DROP DATABASE test_database") +def materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, mysql_node, service_name): + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) + mysql_node.query("CREATE TABLE test_database.test_table_1 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), " + "_decimal DECIMAL(65, 30)) ENGINE = InnoDB;") + mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") + mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(2, '2020-01-01 01:02:03.000000', '2020-01-01 01:02:03.000', ." + ('0' * 29) + "1)") + mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(3, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.99', -" + ('9' * 35) + "." + ('9' * 30) + ")") + mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + "1\t2020-01-01 01:02:03.999999\t2020-01-01 01:02:03.999\t" + ('9' * 35) + "." + ('9' * 30) + "\n" + "2\t2020-01-01 01:02:03.000000\t2020-01-01 01:02:03.000\t0." + ('0' * 29) + "1\n" + "3\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:03.990\t-" + ('9' * 35) + "." + ('9' * 30) + "\n" + "4\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:03.999\t-0." + ('0' * 29) + "1\n") + clickhouse_node.query("DROP DATABASE test_database") + mysql_node.query("DROP DATABASE test_database") + + + def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") From a150b1345bd669dadb0e37e9c6c6a438eebc1b7e Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 8 Sep 2020 11:29:18 +0800 Subject: [PATCH 044/273] ISSUES-4006 add integration test --- tests/integration/test_materialize_mysql_database/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index bfda4e7e840..46ade687b57 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -89,10 +89,13 @@ def started_mysql_8_0(): def test_materialize_database_dml_with_mysql_5_7(started_cluster, started_mysql_5_7): materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql5_7") + materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_5_7, "mysql5_7") def test_materialize_database_dml_with_mysql_8_0(started_cluster, started_mysql_8_0): materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_8_0, "mysql8_0") + def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_5_7): try: From 77b214f7ab18017833c9bae7430319f47864341a Mon Sep 17 00:00:00 2001 From: Gao Qiang <30835199+dreamerfable@users.noreply.github.com> Date: Tue, 8 Sep 2020 22:59:19 +0800 Subject: [PATCH 045/273] Update custom-partitioning-key.md --- .../custom-partitioning-key.md | 70 ++++++++++--------- 1 file changed, 37 insertions(+), 33 deletions(-) diff --git a/docs/zh/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/zh/engines/table-engines/mergetree-family/custom-partitioning-key.md index d7653ca05d6..cf3ac76c8ce 100644 --- a/docs/zh/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/zh/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -2,9 +2,9 @@ [MergeTree](mergetree.md) 系列的表(包括 [可复制表](replication.md) )可以使用分区。基于 MergeTree 表的 [物化视图](../special/materializedview.md#materializedview) 也支持分区。 -一个分区是指按指定规则逻辑组合一起的表的记录集。可以按任意标准进行分区,如按月,按日或按事件类型。为了减少需要操作的数据,每个分区都是分开存储的。访问数据时,ClickHouse 尽量使用这些分区的最小子集。 +分区是在一个表中通过指定的规则划分而成的逻辑数据集。可以按任意标准进行分区,如按月,按日或按事件类型。为了减少需要操作的数据,每个分区都是分开存储的。访问数据时,ClickHouse 尽量使用这些分区的最小子集。 -分区是在 [建表](mergetree.md#table_engine-mergetree-creating-a-table) 的 `PARTITION BY expr` 子句中指定。分区键可以是关于列的任何表达式。例如,指定按月分区,表达式为 `toYYYYMM(date_column)`: +分区是在 [建表](mergetree.md#table_engine-mergetree-creating-a-table) 时通过 `PARTITION BY expr` 子句指定的。分区键可以是表中列的任意表达式。例如,指定按月分区,表达式为 `toYYYYMM(date_column)`: ``` sql CREATE TABLE visits @@ -30,10 +30,10 @@ ORDER BY (CounterID, StartDate, intHash32(UserID)); 新数据插入到表中时,这些数据会存储为按主键排序的新片段(块)。插入后 10-15 分钟,同一分区的各个片段会合并为一整个片段。 -!!! attention "注意" - 那些有相同分区表达式值的数据片段才会合并。这意味着 **你不应该用太精细的分区方案**(超过一千个分区)。否则,会因为文件系统中的文件数量和需要找开的文件描述符过多,导致 `SELECT` 查询效率不佳。 +!!! info "注意" + 那些有相同分区表达式值的数据片段才会合并。这意味着 **你不应该用太精细的分区方案**(超过一千个分区)。否则,会因为文件系统中的文件数量过多和需要打开的文件描述符过多,导致 `SELECT` 查询效率不佳。 -可以通过 [系统。零件](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md#system_tables-parts) 表查看表片段和分区信息。例如,假设我们有一个 `visits` 表,按月分区。对 `system.parts` 表执行 `SELECT`: +可以通过 [system.parts](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md#system_tables-parts) 表查看表片段和分区信息。例如,假设我们有一个 `visits` 表,按月分区。对 `system.parts` 表执行 `SELECT`: ``` sql SELECT @@ -44,55 +44,59 @@ FROM system.parts WHERE table = 'visits' ``` - ┌─partition─┬─name───────────┬─active─┐ - │ 201901 │ 201901_1_3_1 │ 0 │ - │ 201901 │ 201901_1_9_2 │ 1 │ - │ 201901 │ 201901_8_8_0 │ 0 │ - │ 201901 │ 201901_9_9_0 │ 0 │ - │ 201902 │ 201902_4_6_1 │ 1 │ - │ 201902 │ 201902_10_10_0 │ 1 │ - │ 201902 │ 201902_11_11_0 │ 1 │ - └───────────┴────────────────┴────────┘ +``` text +┌─partition─┬─name───────────┬─active─┐ +│ 201901 │ 201901_1_3_1 │ 0 │ +│ 201901 │ 201901_1_9_2 │ 1 │ +│ 201901 │ 201901_8_8_0 │ 0 │ +│ 201901 │ 201901_9_9_0 │ 0 │ +│ 201902 │ 201902_4_6_1 │ 1 │ +│ 201902 │ 201902_10_10_0 │ 1 │ +│ 201902 │ 201902_11_11_0 │ 1 │ +└───────────┴────────────────┴────────┘ +``` `partition` 列存储分区的名称。此示例中有两个分区:`201901` 和 `201902`。在 [ALTER … PARTITION](#alter_manipulations-with-partitions) 语句中你可以使用该列值来指定分区名称。 `name` 列为分区中数据片段的名称。在 [ALTER ATTACH PART](#alter_attach-partition) 语句中你可以使用此列值中来指定片段名称。 -这里我们拆解下第一部分的名称:`201901_1_3_1`: +这里我们拆解下第一个数据片段的名称:`201901_1_3_1`: - `201901` 是分区名称。 - `1` 是数据块的最小编号。 - `3` 是数据块的最大编号。 - `1` 是块级别(即在由块组成的合并树中,该块在树中的深度)。 -!!! attention "注意" +!!! info "注意" 旧类型表的片段名称为:`20190117_20190123_2_2_0`(最小日期 - 最大日期 - 最小块编号 - 最大块编号 - 块级别)。 -`active` 列为片段状态。`1` 激活状态;`0` 非激活状态。非激活片段是那些在合并到较大片段之后剩余的源数据片段。损坏的数据片段也表示为非活动状态。 +`active` 列为片段状态。`1` 代表激活状态;`0` 代表非激活状态。非激活片段是那些在合并到较大片段之后剩余的源数据片段。损坏的数据片段也表示为非活动状态。 -正如在示例中所看到的,同一分区中有几个独立的片段(例如,`201901_1_3_1`和`201901_1_9_2`)。这意味着这些片段尚未合并。ClickHouse 大约在插入后15分钟定期报告合并操作,合并插入的数据片段。此外,你也可以使用 [OPTIMIZE](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md#misc_operations-optimize) 语句直接执行合并。例: +正如在示例中所看到的,同一分区中有几个独立的片段(例如,`201901_1_3_1`和`201901_1_9_2`)。这意味着这些片段尚未合并。ClickHouse 会定期的对插入的数据片段进行合并,大约是在插入后15分钟左右。此外,你也可以使用 [OPTIMIZE](../../../sql-reference/statements/misc.md#misc_operations-optimize) 语句发起一个计划外的合并。例如: ``` sql OPTIMIZE TABLE visits PARTITION 201902; ``` - ┌─partition─┬─name───────────┬─active─┐ - │ 201901 │ 201901_1_3_1 │ 0 │ - │ 201901 │ 201901_1_9_2 │ 1 │ - │ 201901 │ 201901_8_8_0 │ 0 │ - │ 201901 │ 201901_9_9_0 │ 0 │ - │ 201902 │ 201902_4_6_1 │ 0 │ - │ 201902 │ 201902_4_11_2 │ 1 │ - │ 201902 │ 201902_10_10_0 │ 0 │ - │ 201902 │ 201902_11_11_0 │ 0 │ - └───────────┴────────────────┴────────┘ +``` +┌─partition─┬─name───────────┬─active─┐ +│ 201901 │ 201901_1_3_1 │ 0 │ +│ 201901 │ 201901_1_9_2 │ 1 │ +│ 201901 │ 201901_8_8_0 │ 0 │ +│ 201901 │ 201901_9_9_0 │ 0 │ +│ 201902 │ 201902_4_6_1 │ 0 │ +│ 201902 │ 201902_4_11_2 │ 1 │ +│ 201902 │ 201902_10_10_0 │ 0 │ +│ 201902 │ 201902_11_11_0 │ 0 │ +└───────────┴────────────────┴────────┘ +``` -非激活片段会在合并后的10分钟左右删除。 +非激活片段会在合并后的10分钟左右被删除。 查看片段和分区信息的另一种方法是进入表的目录:`/var/lib/clickhouse/data///`。例如: ``` bash -dev:/var/lib/clickhouse/data/default/visits$ ls -l +/var/lib/clickhouse/data/default/visits$ ls -l total 40 drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 1 16:48 201901_1_3_1 drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 16:17 201901_1_9_2 @@ -105,12 +109,12 @@ drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 12:09 201902_4_6_1 drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 1 16:48 detached ``` -文件夹 ‘201901\_1\_1\_0’,‘201901\_1\_7\_1’ 等是片段的目录。每个片段都与一个对应的分区相关,并且只包含这个月的数据(本例中的表按月分区)。 +‘201901\_1\_1\_0’,‘201901\_1\_7\_1’ 等文件夹是数据片段的目录。每个片段都与一个对应的分区相关,并且只包含这个月的数据(本例中的表按月分区)。 -`detached` 目录存放着使用 [DETACH](../../../sql-reference/statements/alter.md#alter_detach-partition) 语句从表中分离的片段。损坏的片段也会移到该目录,而不是删除。服务器不使用`detached`目录中的片段。可以随时添加,删除或修改此目录中的数据 – 在运行 [ATTACH](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md#alter_attach-partition) 语句前,服务器不会感知到。 +`detached` 目录存放着使用 [DETACH](../../../sql-reference/statements/alter.md#alter_detach-partition) 语句从表中卸载的片段。损坏的片段不会被删除而是也会移到该目录下。服务器不会去使用`detached`目录中的数据片段。因此你可以随时添加,删除或修改此目录中的数据 – 在运行 [ATTACH](../../../sql-reference/statements/alter.md#alter_attach-partition) 语句前,服务器不会感知到。 注意,在操作服务器时,你不能手动更改文件系统上的片段集或其数据,因为服务器不会感知到这些修改。对于非复制表,可以在服务器停止时执行这些操作,但不建议这样做。对于复制表,在任何情况下都不要更改片段文件。 -ClickHouse 支持对分区执行这些操作:删除分区,从一个表复制到另一个表,或创建备份。了解分区的所有操作,请参阅 [分区和片段的操作](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md#alter_manipulations-with-partitions) 一节。 +ClickHouse 支持对分区执行这些操作:删除分区,将分区从一个表复制到另一个表,或创建备份。了解分区的所有操作,请参阅 [分区和片段的操作](../../../sql-reference/statements/alter.md#alter_manipulations-with-partitions) 一节。 [来源文章](https://clickhouse.tech/docs/en/operations/table_engines/custom_partitioning_key/) From c4f0465a7c2c7732f5977f05c6cb8f60de554509 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Tue, 8 Sep 2020 18:54:30 +0300 Subject: [PATCH 046/273] limit --- src/Columns/ColumnNullable.cpp | 33 ++++++++++++++++++++++----------- 1 file changed, 22 insertions(+), 11 deletions(-) diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 12934b9420b..27c126a979e 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -334,8 +334,7 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire if (equal_ranges.empty()) return; - if (limit >= equal_ranges.back().second || limit >= size()) - limit = 0; + std::cout << "limit " << limit << std::endl; /// We will sort nested columns into `new_ranges` and call updatePermutation in next columns with `null_ranges`. EqualRanges new_ranges, null_ranges; @@ -347,19 +346,22 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire /// Shift all NULL values to the end. for (const auto & [first, last] : equal_ranges) { + /// Current interval is righter than limit. + if (first > limit) + break; + /// Consider a half interval [first, last) size_t read_idx = first; size_t write_idx = first; size_t end_idx = last; - if (!limit) - limit = end_idx - read_idx; - else - limit = std::min(end_idx - read_idx, limit); + size_t current_limit = end_idx; + if (limit && limit >= read_idx && limit <= end_idx) + current_limit = limit; /// We simply check the limit not to do extra work. /// Since interval begins from `first`, not from zero, we add `first` to the right side of the inequality. - while (read_idx < first + limit && !isNullAt(res[read_idx])) + while (read_idx < current_limit && !isNullAt(res[read_idx])) { ++read_idx; ++write_idx; @@ -377,7 +379,7 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire /// Relative order of NULL elements could be changed, /// but relative order of non-NULLs is preserved. - while (read_idx < end_idx && write_idx < first + limit) + while (read_idx < end_idx && write_idx < current_limit) { if (!isNullAt(res[read_idx])) { @@ -391,7 +393,6 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire if (first != write_idx) new_ranges.emplace_back(first, write_idx); - /// We have a range [write_idx, list) of NULL values if (write_idx != last) null_ranges.emplace_back(write_idx, last); @@ -399,9 +400,12 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire } else { + /// Shift all NULL values to the beginning. for (const auto & [first, last] : equal_ranges) { - /// Shift all NULL values to the beginning. + /// Current interval is righter than limit. + if (first > limit) + break; ssize_t read_idx = last - 1; ssize_t write_idx = last - 1; @@ -429,13 +433,20 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire if (write_idx != static_cast(last)) new_ranges.emplace_back(write_idx + 1, last); - /// We have a range [first, write_idx+1) of NULL values if (static_cast(first) != write_idx) null_ranges.emplace_back(first, write_idx + 1); } } + std::cout << "New Ranges " << std::endl; + for (auto [first, last] : new_ranges ) + std::cout << "first " << first << " last " << last << std::endl; + + std::cout << "Null Ranges " << std::endl; + for (auto [first, last] : null_ranges) + std::cout << "first " << first << " last " << last << std::endl; + getNestedColumn().updatePermutation(reverse, 0, null_direction_hint, res, new_ranges); equal_ranges = std::move(new_ranges); From 614e01b0e46de3f31891b83466f9d784bd19c7bb Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 8 Sep 2020 23:12:55 +0300 Subject: [PATCH 047/273] better --- src/Columns/ColumnNullable.cpp | 16 +++------------- src/Columns/ColumnVector.cpp | 10 +++++++++- 2 files changed, 12 insertions(+), 14 deletions(-) diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 27c126a979e..0b1c306092a 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -334,8 +334,6 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire if (equal_ranges.empty()) return; - std::cout << "limit " << limit << std::endl; - /// We will sort nested columns into `new_ranges` and call updatePermutation in next columns with `null_ranges`. EqualRanges new_ranges, null_ranges; @@ -347,7 +345,7 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire for (const auto & [first, last] : equal_ranges) { /// Current interval is righter than limit. - if (first > limit) + if (limit && first > limit) break; /// Consider a half interval [first, last) @@ -404,7 +402,7 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire for (const auto & [first, last] : equal_ranges) { /// Current interval is righter than limit. - if (first > limit) + if (limit && first > limit) break; ssize_t read_idx = last - 1; @@ -439,15 +437,7 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire } } - std::cout << "New Ranges " << std::endl; - for (auto [first, last] : new_ranges ) - std::cout << "first " << first << " last " << last << std::endl; - - std::cout << "Null Ranges " << std::endl; - for (auto [first, last] : null_ranges) - std::cout << "first " << first << " last " << last << std::endl; - - getNestedColumn().updatePermutation(reverse, 0, null_direction_hint, res, new_ranges); + getNestedColumn().updatePermutation(reverse, limit, null_direction_hint, res, new_ranges); equal_ranges = std::move(new_ranges); std::move(null_ranges.begin(), null_ranges.end(), std::back_inserter(equal_ranges)); diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 416123af8f0..d950c03d49b 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -18,6 +18,8 @@ #include #include +#include + #if !defined(ARCADIA_BUILD) # include # if USE_OPENCL @@ -250,6 +252,7 @@ void ColumnVector::updatePermutation(bool reverse, size_t limit, int nan_dire limit = 0; EqualRanges new_ranges; + SCOPE_EXIT({equal_range = std::move(new_ranges);}); for (size_t i = 0; i < equal_range.size() - bool(limit); ++i) { @@ -278,6 +281,12 @@ void ColumnVector::updatePermutation(bool reverse, size_t limit, int nan_dire if (limit) { const auto & [first, last] = equal_range.back(); + + if (limit < first || limit >= last) + return; + + /// Since then, we are working inside the interval. + if (reverse) std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, greater(*this, nan_direction_hint)); else @@ -310,7 +319,6 @@ void ColumnVector::updatePermutation(bool reverse, size_t limit, int nan_dire new_ranges.emplace_back(new_first, new_last); } } - equal_range = std::move(new_ranges); } template From 62ef728b2f20fed99b43afabeeca451807f4879d Mon Sep 17 00:00:00 2001 From: Winter Zhang Date: Wed, 9 Sep 2020 10:52:14 +0800 Subject: [PATCH 048/273] ISSUES-4006 try fix test failure --- .../test_materialize_mysql_database/materialize_with_ddl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 00b6a87cb37..fa31c43e6c1 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -110,7 +110,7 @@ def materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, mysql_ "1\t2020-01-01 01:02:03.999999\t2020-01-01 01:02:03.999\t" + ('9' * 35) + "." + ('9' * 30) + "\n" "2\t2020-01-01 01:02:03.000000\t2020-01-01 01:02:03.000\t0." + ('0' * 29) + "1\n" "3\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:03.990\t-" + ('9' * 35) + "." + ('9' * 30) + "\n" - "4\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:03.999\t-0." + ('0' * 29) + "1\n") + "4\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:04.000\t-0." + ('0' * 29) + "1\n") clickhouse_node.query("DROP DATABASE test_database") mysql_node.query("DROP DATABASE test_database") From 406f384a45bce0a728e4e8cad06fd34b1577154f Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Wed, 9 Sep 2020 11:24:47 +0800 Subject: [PATCH 049/273] support ILIKE, and separate like,limit function. --- .../InterpreterShowTablesQuery.cpp | 18 ++++- src/Parsers/ASTShowTablesQuery.cpp | 66 ++++++++----------- src/Parsers/ASTShowTablesQuery.h | 2 + 3 files changed, 44 insertions(+), 42 deletions(-) diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index 09c617e12ec..ef7fd840ac5 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -37,7 +37,11 @@ String InterpreterShowTablesQuery::getRewrittenQuery() if (!query.like.empty()) { - rewritten_query << " WHERE name " << (query.not_like ? "NOT " : "") << "LIKE " << std::quoted(query.like, '\''); + rewritten_query + << " WHERE name " + << (query.not_like ? "NOT " : "") + << (query.case_insensitive_like ? "ILIKE " : "LIKE ") + << std::quoted(query.like, '\''); } if (query.limit_length) @@ -54,7 +58,11 @@ String InterpreterShowTablesQuery::getRewrittenQuery() if (!query.like.empty()) { - rewritten_query << " WHERE cluster " << (query.not_like ? "NOT " : "") << "LIKE " << std::quoted(query.like, '\''); + rewritten_query + << " WHERE cluster " + << (query.not_like ? "NOT " : "") + << (query.case_insensitive_like ? "ILIKE " : "LIKE ") + << std::quoted(query.like, '\''); } if (query.limit_length) @@ -98,7 +106,11 @@ String InterpreterShowTablesQuery::getRewrittenQuery() rewritten_query << "database = " << std::quoted(database, '\''); if (!query.like.empty()) - rewritten_query << " AND name " << (query.not_like ? "NOT " : "") << "LIKE " << std::quoted(query.like, '\''); + rewritten_query + << " AND name " + << (query.not_like ? "NOT " : "") + << (query.case_insensitive_like ? "ILIKE " : "LIKE ") + << std::quoted(query.like, '\''); else if (query.where_expression) rewritten_query << " AND (" << query.where_expression << ")"; diff --git a/src/Parsers/ASTShowTablesQuery.cpp b/src/Parsers/ASTShowTablesQuery.cpp index ce44d2b56de..b59ba07d03e 100644 --- a/src/Parsers/ASTShowTablesQuery.cpp +++ b/src/Parsers/ASTShowTablesQuery.cpp @@ -13,43 +13,41 @@ ASTPtr ASTShowTablesQuery::clone() const return res; } +void ASTShowTablesQuery::formatLike(const FormatSettings & settings) const +{ + if (!like.empty()) + settings.ostr + << (settings.hilite ? hilite_keyword : "") + << (not_like ? " NOT" : "") + << (case_insensitive_like ? " ILIKE " : " LIKE ") + << (settings.hilite ? hilite_none : "") + << std::quoted(like, '\''); +} + +void ASTShowTablesQuery::formatLimit(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + if (limit_length) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " LIMIT " << (settings.hilite ? hilite_none : ""); + limit_length->formatImpl(settings, state, frame); + } +} + void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { if (databases) { settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW DATABASES" << (settings.hilite ? hilite_none : ""); + formatLike(settings); + formatLimit(settings, state, frame); - if (!like.empty()) - settings.ostr - << (settings.hilite ? hilite_keyword : "") - << (not_like ? " NOT" : "") - << (case_insensitive_like ? " ILIKE " : " LIKE ") - << (settings.hilite ? hilite_none : "") - << std::quoted(like, '\''); - - if (limit_length) - { - settings.ostr << (settings.hilite ? hilite_keyword : "") << " LIMIT " << (settings.hilite ? hilite_none : ""); - limit_length->formatImpl(settings, state, frame); - } } else if (clusters) { settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW CLUSTERS" << (settings.hilite ? hilite_none : ""); + formatLike(settings); + formatLimit(settings, state, frame); - if (!like.empty()) - settings.ostr - << (settings.hilite ? hilite_keyword : "") - << (not_like ? " NOT" : "") - << (case_insensitive_like ? " ILIKE " : " LIKE ") - << (settings.hilite ? hilite_none : "") - << std::quoted(like, '\''); - - if (limit_length) - { - settings.ostr << (settings.hilite ? hilite_keyword : "") << " LIMIT " << (settings.hilite ? hilite_none : ""); - limit_length->formatImpl(settings, state, frame); - } } else if (cluster) { @@ -65,25 +63,15 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(from); - if (!like.empty()) - settings.ostr - << (settings.hilite ? hilite_keyword : "") - << (not_like ? " NOT" : "") - << (case_insensitive_like ? " ILIKE " : " LIKE ") - << (settings.hilite ? hilite_none : "") - << std::quoted(like, '\''); + formatLike(settings); - else if (where_expression) + if (where_expression) { settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : ""); where_expression->formatImpl(settings, state, frame); } - if (limit_length) - { - settings.ostr << (settings.hilite ? hilite_keyword : "") << " LIMIT " << (settings.hilite ? hilite_none : ""); - limit_length->formatImpl(settings, state, frame); - } + formatLimit(settings, state, frame); } } diff --git a/src/Parsers/ASTShowTablesQuery.h b/src/Parsers/ASTShowTablesQuery.h index acf365be91a..43976e8a958 100644 --- a/src/Parsers/ASTShowTablesQuery.h +++ b/src/Parsers/ASTShowTablesQuery.h @@ -36,6 +36,8 @@ public: ASTPtr clone() const override; protected: + void formatLike(const FormatSettings & settings) const; + void formatLimit(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const; void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; }; From c03a9487ca542b93fa51db1a671f1f365da0c081 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 9 Sep 2020 14:55:20 +0300 Subject: [PATCH 050/273] another test + fix --- src/Columns/ColumnNullable.cpp | 11 ++--- .../01457_order_by_limit.reference | 40 +++++++++++++++++++ .../0_stateless/01457_order_by_limit.sql | 30 ++++++++++++++ 3 files changed, 73 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/01457_order_by_limit.reference create mode 100644 tests/queries/0_stateless/01457_order_by_limit.sql diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 0b1c306092a..bdbc941c1e7 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -353,13 +353,8 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire size_t write_idx = first; size_t end_idx = last; - size_t current_limit = end_idx; - if (limit && limit >= read_idx && limit <= end_idx) - current_limit = limit; - - /// We simply check the limit not to do extra work. - /// Since interval begins from `first`, not from zero, we add `first` to the right side of the inequality. - while (read_idx < current_limit && !isNullAt(res[read_idx])) + /// We can't check the limit here because the interval is not sorted by nested column. + while (read_idx < end_idx && !isNullAt(res[read_idx])) { ++read_idx; ++write_idx; @@ -377,7 +372,7 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire /// Relative order of NULL elements could be changed, /// but relative order of non-NULLs is preserved. - while (read_idx < end_idx && write_idx < current_limit) + while (read_idx < end_idx && write_idx < end_idx) { if (!isNullAt(res[read_idx])) { diff --git a/tests/queries/0_stateless/01457_order_by_limit.reference b/tests/queries/0_stateless/01457_order_by_limit.reference new file mode 100644 index 00000000000..348bc7ad1d4 --- /dev/null +++ b/tests/queries/0_stateless/01457_order_by_limit.reference @@ -0,0 +1,40 @@ +asc nulls last, asc +1 1 +1 2 +1 3 +1 4 +asc nulls first, asc +1 1 +1 2 +1 3 +1 4 +desc nulls last, asc +1 1 +1 2 +1 3 +1 4 +desc nulls first, asc +1 1 +1 2 +1 3 +1 4 +asc nulls last, desc +1 8 +1 7 +1 6 +1 5 +asc nulls first, desc +1 8 +1 7 +1 6 +1 5 +desc nulls last, desc +1 8 +1 7 +1 6 +1 5 +desc nulls first, desc +1 8 +1 7 +1 6 +1 5 diff --git a/tests/queries/0_stateless/01457_order_by_limit.sql b/tests/queries/0_stateless/01457_order_by_limit.sql new file mode 100644 index 00000000000..514aaeac4ab --- /dev/null +++ b/tests/queries/0_stateless/01457_order_by_limit.sql @@ -0,0 +1,30 @@ +drop table if exists order_by_another; + +create table order_by_another (a Nullable(UInt64), b UInt64) Engine = MergeTree order by tuple(); +insert into order_by_another values (1, 8), (1, 7), (1, 6), (1, 5), (1, 4), (1, 3), (1, 2), (1, 1); + +select 'asc nulls last, asc'; +select a, b from order_by_another order by a asc nulls last, b asc limit 4; + +select 'asc nulls first, asc'; +select a, b from order_by_another order by a asc nulls first, b asc limit 4; + +select 'desc nulls last, asc'; +select a, b from order_by_another order by a desc nulls last, b asc limit 4; + +select 'desc nulls first, asc'; +select a, b from order_by_another order by a desc nulls first, b asc limit 4; + +select 'asc nulls last, desc'; +select a, b from order_by_another order by a asc nulls last, b desc limit 4; + +select 'asc nulls first, desc'; +select a, b from order_by_another order by a asc nulls first, b desc limit 4; + +select 'desc nulls last, desc'; +select a, b from order_by_another order by a desc nulls last, b desc limit 4; + +select 'desc nulls first, desc'; +select a, b from order_by_another order by a desc nulls first, b desc limit 4; + +drop table if exists order_by_another; \ No newline at end of file From 063bcf6ff73d620550cbb2e2f77a7519abdcd665 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=9D=B3=E9=98=B3?= <260893248@qq.com> Date: Wed, 9 Sep 2020 20:03:26 +0800 Subject: [PATCH 051/273] =?UTF-8?q?"=E5=9B=9E=E8=AF=9D"=20->=20"=E7=9A=84?= =?UTF-8?q?=E4=BC=9A=E8=AF=9D"?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit fix typo in Chinese --- docs/zh/introduction/history.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/introduction/history.md b/docs/zh/introduction/history.md index 1871bd75588..29c8c263f9f 100644 --- a/docs/zh/introduction/history.md +++ b/docs/zh/introduction/history.md @@ -13,7 +13,7 @@ Yandex.Metrica基于用户定义的字段,对实时访问、连接会话,生 ClickHouse还被使用在: -- 存储来自Yandex.Metrica回话重放数据。 +- 存储来自Yandex.Metrica的会话重放数据。 - 处理中间数据 - 与Analytics一起构建全球报表。 - 为调试Yandex.Metrica引擎运行查询 From 530057e79f9dad7467d9879f4d8f4420768af669 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 9 Sep 2020 15:27:05 +0300 Subject: [PATCH 052/273] update all columns --- src/Columns/ColumnDecimal.cpp | 28 +++++++++++++------ src/Columns/ColumnFixedString.cpp | 28 +++++++++++++------ src/Columns/ColumnLowCardinality.cpp | 28 +++++++++++++------ src/Columns/ColumnString.cpp | 42 +++++++++++++++++----------- src/Columns/ColumnTuple.cpp | 15 +++++----- src/Columns/ColumnUnique.h | 12 ++++---- src/Columns/ColumnVector.cpp | 6 ++-- 7 files changed, 99 insertions(+), 60 deletions(-) diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 6bab4228e9d..a899099ca4f 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -7,6 +7,7 @@ #include #include +#include #include @@ -142,28 +143,31 @@ void ColumnDecimal::getPermutation(bool reverse, size_t limit, int , IColumn: } template -void ColumnDecimal::updatePermutation(bool reverse, size_t limit, int, IColumn::Permutation & res, EqualRanges & equal_range) const +void ColumnDecimal::updatePermutation(bool reverse, size_t limit, int, IColumn::Permutation & res, EqualRanges & equal_ranges) const { - if (equal_range.empty()) + if (equal_ranges.empty()) return; - if (limit >= data.size() || limit >= equal_range.back().second) + if (limit >= data.size() || limit >= equal_ranges.back().second) limit = 0; - size_t n = equal_range.size(); + size_t number_of_ranges = equal_ranges.size(); if (limit) - --n; + --number_of_ranges; EqualRanges new_ranges; - for (size_t i = 0; i < n; ++i) + SCOPE_EXIT({equal_ranges = std::move(new_ranges);}); + + for (size_t i = 0; i < number_of_ranges; ++i) { - const auto& [first, last] = equal_range[i]; + const auto& [first, last] = equal_ranges[i]; if (reverse) std::partial_sort(res.begin() + first, res.begin() + last, res.begin() + last, [this](size_t a, size_t b) { return data[a] > data[b]; }); else std::partial_sort(res.begin() + first, res.begin() + last, res.begin() + last, [this](size_t a, size_t b) { return data[a] < data[b]; }); + auto new_first = first; for (auto j = first + 1; j < last; ++j) { @@ -181,13 +185,20 @@ void ColumnDecimal::updatePermutation(bool reverse, size_t limit, int, IColum if (limit) { - const auto& [first, last] = equal_range.back(); + const auto& [first, last] = equal_ranges.back(); + + if (limit < first || limit >= last) + return; + + /// Since then we are working inside the interval. + if (reverse) std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this](size_t a, size_t b) { return data[a] > data[b]; }); else std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this](size_t a, size_t b) { return data[a] < data[b]; }); + auto new_first = first; for (auto j = first + 1; j < limit; ++j) { @@ -211,7 +222,6 @@ void ColumnDecimal::updatePermutation(bool reverse, size_t limit, int, IColum if (new_last - new_first > 1) new_ranges.emplace_back(new_first, new_last); } - equal_range = std::move(new_ranges); } template diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index c10caa37b28..9188a988198 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -9,6 +9,8 @@ #include #include +#include + #include #include @@ -168,27 +170,29 @@ void ColumnFixedString::getPermutation(bool reverse, size_t limit, int /*nan_dir } } -void ColumnFixedString::updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_range) const +void ColumnFixedString::updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const { - if (equal_range.empty()) + if (equal_ranges.empty()) return; - if (limit >= size() || limit >= equal_range.back().second) + if (limit >= size() || limit >= equal_ranges.back().second) limit = 0; - size_t k = equal_range.size(); + size_t number_of_ranges = equal_ranges.size(); if (limit) - --k; + --number_of_ranges; EqualRanges new_ranges; + SCOPE_EXIT({equal_ranges = std::move(new_ranges);}); - for (size_t i = 0; i < k; ++i) + for (size_t i = 0; i < number_of_ranges; ++i) { - const auto& [first, last] = equal_range[i]; + const auto& [first, last] = equal_ranges[i]; if (reverse) std::sort(res.begin() + first, res.begin() + last, less(*this)); else std::sort(res.begin() + first, res.begin() + last, less(*this)); + auto new_first = first; for (auto j = first + 1; j < last; ++j) { @@ -205,11 +209,18 @@ void ColumnFixedString::updatePermutation(bool reverse, size_t limit, int, Permu } if (limit) { - const auto& [first, last] = equal_range.back(); + const auto& [first, last] = equal_ranges.back(); + + if (limit < first || limit >= last) + return; + + /// Since then we are working inside the interval. + if (reverse) std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); else std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); + auto new_first = first; for (auto j = first + 1; j < limit; ++j) { @@ -233,7 +244,6 @@ void ColumnFixedString::updatePermutation(bool reverse, size_t limit, int, Permu if (new_last - new_first > 1) new_ranges.emplace_back(new_first, new_last); } - equal_range = std::move(new_ranges); } void ColumnFixedString::insertRangeFrom(const IColumn & src, size_t start, size_t length) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 0613e5e2b71..b6f1aede190 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -6,6 +6,7 @@ #include #include +#include namespace DB { @@ -329,22 +330,24 @@ void ColumnLowCardinality::getPermutation(bool reverse, size_t limit, int nan_di } } -void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_range) const +void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const { - if (equal_range.empty()) + if (equal_ranges.empty()) return; - if (limit >= size() || limit >= equal_range.back().second) + if (limit >= size() || limit >= equal_ranges.back().second) limit = 0; - size_t n = equal_range.size(); + size_t number_of_ranges = equal_ranges.size(); if (limit) - --n; + --number_of_ranges; EqualRanges new_ranges; - for (size_t i = 0; i < n; ++i) + SCOPE_EXIT({equal_ranges = std::move(new_ranges);}); + + for (size_t i = 0; i < number_of_ranges; ++i) { - const auto& [first, last] = equal_range[i]; + const auto& [first, last] = equal_ranges[i]; if (reverse) std::sort(res.begin() + first, res.begin() + last, [this, nan_direction_hint](size_t a, size_t b) {return getDictionary().compareAt(getIndexes().getUInt(a), getIndexes().getUInt(b), getDictionary(), nan_direction_hint) > 0; }); @@ -369,7 +372,13 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan if (limit) { - const auto& [first, last] = equal_range.back(); + const auto& [first, last] = equal_ranges.back(); + + if (limit < first || limit >= last) + return; + + /// Since then we are working inside the interval. + if (reverse) std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this, nan_direction_hint](size_t a, size_t b) {return getDictionary().compareAt(getIndexes().getUInt(a), getIndexes().getUInt(b), getDictionary(), nan_direction_hint) > 0; }); @@ -377,6 +386,7 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this, nan_direction_hint](size_t a, size_t b) {return getDictionary().compareAt(getIndexes().getUInt(a), getIndexes().getUInt(b), getDictionary(), nan_direction_hint) < 0; }); auto new_first = first; + for (auto j = first + 1; j < limit; ++j) { if (getDictionary().compareAt(getIndexes().getUInt(new_first), getIndexes().getUInt(j), getDictionary(), nan_direction_hint) != 0) @@ -387,6 +397,7 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan new_first = j; } } + auto new_last = limit; for (auto j = limit; j < last; ++j) { @@ -399,7 +410,6 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan if (new_last - new_first > 1) new_ranges.emplace_back(new_first, new_last); } - equal_range = std::move(new_ranges); } std::vector ColumnLowCardinality::scatter(ColumnIndex num_columns, const Selector & selector) const diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 57795535a64..449465e7146 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -9,7 +9,7 @@ #include #include - +#include namespace DB { @@ -325,28 +325,30 @@ void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_directio } } -void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res, EqualRanges & equal_range) const +void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res, EqualRanges & equal_ranges) const { - if (equal_range.empty()) + if (equal_ranges.empty()) return; - if (limit >= size() || limit > equal_range.back().second) + if (limit >= size() || limit > equal_ranges.back().second) limit = 0; EqualRanges new_ranges; - auto less_true = less(*this); - auto less_false = less(*this); - size_t n = equal_range.size(); - if (limit) - --n; + SCOPE_EXIT({equal_ranges = std::move(new_ranges);}); - for (size_t i = 0; i < n; ++i) + size_t number_of_ranges = equal_ranges.size(); + if (limit) + --number_of_ranges; + + for (size_t i = 0; i < number_of_ranges; ++i) { - const auto &[first, last] = equal_range[i]; + const auto & [first, last] = equal_ranges[i]; + if (reverse) - std::sort(res.begin() + first, res.begin() + last, less_false); + std::sort(res.begin() + first, res.begin() + last, less(*this)); else - std::sort(res.begin() + first, res.begin() + last, less_true); + std::sort(res.begin() + first, res.begin() + last, less(*this)); + size_t new_first = first; for (size_t j = first + 1; j < last; ++j) { @@ -366,11 +368,18 @@ void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direc if (limit) { - const auto &[first, last] = equal_range.back(); + const auto & [first, last] = equal_ranges.back(); + + if (limit < first || limit >= last) + return; + + /// Since then we are working inside the interval. + if (reverse) - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less_false); + std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less_true); + std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); + size_t new_first = first; for (size_t j = first + 1; j < limit; ++j) { @@ -397,7 +406,6 @@ void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direc if (new_last - new_first > 1) new_ranges.emplace_back(new_first, new_last); } - equal_range = std::move(new_ranges); } ColumnPtr ColumnString::replicate(const Offsets & replicate_offsets) const diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 09c7472b22b..98a6611edb7 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -344,18 +344,19 @@ void ColumnTuple::getPermutation(bool reverse, size_t limit, int nan_direction_h } } -void ColumnTuple::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_range) const +void ColumnTuple::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const { - if (equal_range.empty()) + if (equal_ranges.empty()) return; - for (const auto& column : columns) + for (const auto & column : columns) { - column->updatePermutation(reverse, limit, nan_direction_hint, res, equal_range); - while (limit && !equal_range.empty() && limit <= equal_range.back().first) - equal_range.pop_back(); + column->updatePermutation(reverse, limit, nan_direction_hint, res, equal_ranges); - if (equal_range.empty()) + while (limit && !equal_ranges.empty() && limit <= equal_ranges.back().first) + equal_ranges.pop_back(); + + if (equal_ranges.empty()) break; } } diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index 59febe52112..c453af78bb1 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -387,15 +387,15 @@ int ColumnUnique::compareAt(size_t n, size_t m, const IColumn & rhs, } template -void ColumnUnique::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_range) const +void ColumnUnique::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const { - if (equal_range.empty()) + if (equal_ranges.empty()) return; bool found_null_value_index = false; - for (size_t i = 0; i < equal_range.size() && !found_null_value_index; ++i) + for (size_t i = 0; i < equal_ranges.size() && !found_null_value_index; ++i) { - auto& [first, last] = equal_range[i]; + auto & [first, last] = equal_ranges[i]; for (auto j = first; j < last; ++j) { if (res[j] == getNullValueIndex()) @@ -412,14 +412,14 @@ void ColumnUnique::updatePermutation(bool reverse, size_t limit, int } if (last - first <= 1) { - equal_range.erase(equal_range.begin() + i); + equal_ranges.erase(equal_ranges.begin() + i); } found_null_value_index = true; break; } } } - getNestedColumn()->updatePermutation(reverse, limit, nan_direction_hint, res, equal_range); + getNestedColumn()->updatePermutation(reverse, limit, nan_direction_hint, res, equal_ranges); } template diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index d950c03d49b..9582aa7278a 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -15,10 +15,10 @@ #include #include #include -#include -#include - #include +#include + + #if !defined(ARCADIA_BUILD) # include From f04d67b6888421d4a84c023fcbadc3779ac180fe Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 9 Sep 2020 15:41:51 +0300 Subject: [PATCH 053/273] better --- src/Columns/ColumnLowCardinality.cpp | 2 +- src/Columns/ColumnString.cpp | 26 +++++++++++++++++--------- src/Columns/ColumnUnique.h | 2 +- src/Interpreters/sortBlock.cpp | 8 ++------ 4 files changed, 21 insertions(+), 17 deletions(-) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index b6f1aede190..66a1a009ae9 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -372,7 +372,7 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan if (limit) { - const auto& [first, last] = equal_ranges.back(); + const auto & [first, last] = equal_ranges.back(); if (limit < first || limit >= last) return; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 449465e7146..b714776aa04 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -545,19 +545,22 @@ void ColumnString::getPermutationWithCollation(const Collator & collator, bool r } } -void ColumnString::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation &res, EqualRanges &equal_range) const +void ColumnString::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const { - if (limit >= size() || limit >= equal_range.back().second) + if (limit >= size() || limit >= equal_ranges.back().second) limit = 0; - size_t n = equal_range.size(); + size_t number_of_ranges = equal_ranges.size(); if (limit) - --n; + --number_of_ranges; EqualRanges new_ranges; - for (size_t i = 0; i < n; ++i) + SCOPE_EXIT({equal_ranges = std::move(new_ranges);}); + + for (size_t i = 0; i < number_of_ranges; ++i) { - const auto& [first, last] = equal_range[i]; + const auto& [first, last] = equal_ranges[i]; + if (reverse) std::sort(res.begin() + first, res.begin() + last, lessWithCollation(*this, collator)); else @@ -577,16 +580,22 @@ void ColumnString::updatePermutationWithCollation(const Collator & collator, boo } if (last - new_first > 1) new_ranges.emplace_back(new_first, last); - } if (limit) { - const auto& [first, last] = equal_range.back(); + const auto & [first, last] = equal_ranges.back(); + + if (limit < first || limit >= last) + return; + + /// Since then we are working inside the interval. + if (reverse) std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, lessWithCollation(*this, collator)); else std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, lessWithCollation(*this, collator)); + auto new_first = first; for (auto j = first + 1; j < limit; ++j) { @@ -614,7 +623,6 @@ void ColumnString::updatePermutationWithCollation(const Collator & collator, boo if (new_last - new_first > 1) new_ranges.emplace_back(new_first, new_last); } - equal_range = std::move(new_ranges); } void ColumnString::protect() diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index c453af78bb1..d87fdd65d15 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -382,7 +382,7 @@ int ColumnUnique::compareAt(size_t n, size_t m, const IColumn & rhs, } } - auto & column_unique = static_cast(rhs); + const auto & column_unique = static_cast(rhs); return getNestedColumn()->compareAt(n, m, *column_unique.getNestedColumn(), nan_direction_hint); } diff --git a/src/Interpreters/sortBlock.cpp b/src/Interpreters/sortBlock.cpp index cb3c36e5356..d84708b9c57 100644 --- a/src/Interpreters/sortBlock.cpp +++ b/src/Interpreters/sortBlock.cpp @@ -211,13 +211,11 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) for (const auto & column : columns_with_sort_desc) { while (!ranges.empty() && limit && limit <= ranges.back().first) - { ranges.pop_back(); - } + if (ranges.empty()) - { break; - } + column.column->updatePermutation( column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); } @@ -225,9 +223,7 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) size_t columns = block.columns(); for (size_t i = 0; i < columns; ++i) - { block.getByPosition(i).column = block.getByPosition(i).column->permute(perm, limit); - } } } From ead6bfe05cbd6ce134ddd0ed370702f96ed864e7 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 9 Sep 2020 15:42:56 +0300 Subject: [PATCH 054/273] better[2] --- src/Columns/ColumnString.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index b714776aa04..a3da1d745c1 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -547,6 +547,9 @@ void ColumnString::getPermutationWithCollation(const Collator & collator, bool r void ColumnString::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const { + if (equal_ranges.empty()) + return; + if (limit >= size() || limit >= equal_ranges.back().second) limit = 0; From 751e4109152671ece957a0c36af43f0e909f41d7 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Wed, 9 Sep 2020 17:11:08 +0300 Subject: [PATCH 055/273] better[3] --- src/Columns/ColumnArray.cpp | 16 +++++++++++----- src/Columns/ColumnDecimal.cpp | 4 ++-- src/Columns/ColumnFixedString.cpp | 4 ++-- src/Columns/ColumnLowCardinality.cpp | 2 +- src/Columns/ColumnString.cpp | 4 ++-- src/Columns/ColumnVector.cpp | 2 +- 6 files changed, 19 insertions(+), 13 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index cd4aa57c18f..e4d17c586ac 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -787,15 +787,15 @@ void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_directio if (limit >= size() || limit >= equal_range.back().second) limit = 0; - size_t n = equal_range.size(); + size_t number_of_ranges = equal_range.size(); if (limit) - --n; + --number_of_ranges; EqualRanges new_ranges; - for (size_t i = 0; i < n; ++i) + for (size_t i = 0; i < number_of_ranges; ++i) { - const auto& [first, last] = equal_range[i]; + const auto & [first, last] = equal_range[i]; if (reverse) std::sort(res.begin() + first, res.begin() + last, Less(*this, nan_direction_hint)); @@ -820,7 +820,13 @@ void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_directio if (limit) { - const auto& [first, last] = equal_range.back(); + const auto & [first, last] = equal_range.back(); + + if (limit < first || limit > last) + return; + + /// Since then we are working inside the interval. + if (reverse) std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, Less(*this, nan_direction_hint)); else diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index a899099ca4f..13c7ea46fbc 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -185,9 +185,9 @@ void ColumnDecimal::updatePermutation(bool reverse, size_t limit, int, IColum if (limit) { - const auto& [first, last] = equal_ranges.back(); + const auto & [first, last] = equal_ranges.back(); - if (limit < first || limit >= last) + if (limit < first || limit > last) return; /// Since then we are working inside the interval. diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 9188a988198..0e44b83791c 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -209,9 +209,9 @@ void ColumnFixedString::updatePermutation(bool reverse, size_t limit, int, Permu } if (limit) { - const auto& [first, last] = equal_ranges.back(); + const auto & [first, last] = equal_ranges.back(); - if (limit < first || limit >= last) + if (limit < first || limit > last) return; /// Since then we are working inside the interval. diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 66a1a009ae9..dd1e2f60399 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -374,7 +374,7 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan { const auto & [first, last] = equal_ranges.back(); - if (limit < first || limit >= last) + if (limit < first || limit > last) return; /// Since then we are working inside the interval. diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index a3da1d745c1..23414626a59 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -370,7 +370,7 @@ void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direc { const auto & [first, last] = equal_ranges.back(); - if (limit < first || limit >= last) + if (limit < first || limit > last) return; /// Since then we are working inside the interval. @@ -589,7 +589,7 @@ void ColumnString::updatePermutationWithCollation(const Collator & collator, boo { const auto & [first, last] = equal_ranges.back(); - if (limit < first || limit >= last) + if (limit < first || limit > last) return; /// Since then we are working inside the interval. diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 9582aa7278a..c548ce3ca5c 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -282,7 +282,7 @@ void ColumnVector::updatePermutation(bool reverse, size_t limit, int nan_dire { const auto & [first, last] = equal_range.back(); - if (limit < first || limit >= last) + if (limit < first || limit > last) return; /// Since then, we are working inside the interval. From b8d28c864c8902c2e2aa270112f3308d90a2a490 Mon Sep 17 00:00:00 2001 From: Gao Qiang <30835199+dreamerfable@users.noreply.github.com> Date: Wed, 9 Sep 2020 22:45:49 +0800 Subject: [PATCH 056/273] Update aggregatingmergetree.md --- .../mergetree-family/aggregatingmergetree.md | 22 +++++++++++-------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/docs/zh/engines/table-engines/mergetree-family/aggregatingmergetree.md b/docs/zh/engines/table-engines/mergetree-family/aggregatingmergetree.md index e931b6f6710..03825a41f95 100644 --- a/docs/zh/engines/table-engines/mergetree-family/aggregatingmergetree.md +++ b/docs/zh/engines/table-engines/mergetree-family/aggregatingmergetree.md @@ -1,12 +1,15 @@ # AggregatingMergeTree {#aggregatingmergetree} -该引擎继承自 [MergeTree](mergetree.md),并改变了数据片段的合并逻辑。 ClickHouse 会将相同主键的所有行(在一个数据片段内)替换为单个存储一系列聚合函数状态的行。 +该引擎继承自 [MergeTree](mergetree.md),并改变了数据片段的合并逻辑。 ClickHouse 会将一个数据片段内所有具有相同主键(准确的说是 [排序键](../../../engines/table-engines/mergetree-family/mergetree.md))的行替换成一行,这一行会存储一系列聚合函数的状态。 -可以使用 `AggregatingMergeTree` 表来做增量数据统计聚合,包括物化视图的数据聚合。 +可以使用 `AggregatingMergeTree` 表来做增量数据的聚合统计,包括物化视图的数据聚合。 -引擎需使用 [AggregateFunction](../../../engines/table-engines/mergetree-family/aggregatingmergetree.md) 类型来处理所有列。 +引擎使用以下类型来处理所有列: -如果要按一组规则来合并减少行数,则使用 `AggregatingMergeTree` 是合适的。 +- [AggregateFunction](../../../sql-reference/data-types/aggregatefunction.md) +- [SimpleAggregateFunction](../../../sql-reference/data-types/simpleaggregatefunction.md) + +`AggregatingMergeTree` 适用于能够按照一定的规则缩减行数的情况。 ## 建表 {#jian-biao} @@ -20,10 +23,11 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [PARTITION BY expr] [ORDER BY expr] [SAMPLE BY expr] +[TTL expr] [SETTINGS name=value, ...] ``` -语句参数的说明,请参阅 [语句描述](../../../engines/table-engines/mergetree-family/aggregatingmergetree.md)。 +语句参数的说明,请参阅 [建表语句描述](../../../sql-reference/statements/create.md#create-table-query)。 **子句** @@ -33,7 +37,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] 已弃用的建表方法 -!!! 注意 "注意" +!!! attention "注意" 不要在新项目中使用该方法,可能的话,请将旧项目切换到上述方法。 ``` sql @@ -45,15 +49,15 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ) ENGINE [=] AggregatingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity) ``` -上面的所有参数跟 `MergeTree` 中的一样。 +上面的所有参数的含义跟 `MergeTree` 中的一样。 ## SELECT 和 INSERT {#select-he-insert} -插入数据,需使用带有聚合 -State- 函数的 [INSERT SELECT](../../../engines/table-engines/mergetree-family/aggregatingmergetree.md) 语句。 +要插入数据,需使用带有 -State- 聚合函数的 [INSERT SELECT](../../../sql-reference/statements/insert-into.md) 语句。 从 `AggregatingMergeTree` 表中查询数据时,需使用 `GROUP BY` 子句并且要使用与插入时相同的聚合函数,但后缀要改为 `-Merge` 。 -在 `SELECT` 查询的结果中,对于 ClickHouse 的所有输出格式 `AggregateFunction` 类型的值都实现了特定的二进制表示法。如果直接用 `SELECT` 导出这些数据,例如如用 `TabSeparated` 格式,那么这些导出数据也能直接用 `INSERT` 语句加载导入。 +对于 `SELECT` 查询的结果, `AggregateFunction` 类型的值对 ClickHouse 的所有输出格式都实现了特定的二进制表示法。在进行数据转储时,例如使用 `TabSeparated` 格式进行 `SELECT` 查询,那么这些转储数据也能直接用 `INSERT` 语句导回。 ## 聚合物化视图的示例 {#ju-he-wu-hua-shi-tu-de-shi-li} From 6ed3f7fb86b19b94fa236e5c2659ea7a217357b3 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 9 Sep 2020 23:42:03 +0800 Subject: [PATCH 057/273] ISSUES-4006 try fix integration test failure when mysql 5.7 --- tests/integration/test_materialize_mysql_database/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index c6b3810fc74..7affa665cd3 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -89,7 +89,7 @@ def started_mysql_8_0(): def test_materialize_database_dml_with_mysql_5_7(started_cluster, started_mysql_5_7): materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_5_7, "mysql5_7") + materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_5_7, "mysql1") def test_materialize_database_dml_with_mysql_8_0(started_cluster, started_mysql_8_0): From e91d120e1a9c8326c895fcb45c9e43f5cdedfdd1 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Wed, 9 Sep 2020 20:11:23 +0300 Subject: [PATCH 058/273] Bump CI. From b9bf67b6ac31741246dce7790ec3890fd599ff7d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 10 Sep 2020 04:27:36 +0300 Subject: [PATCH 059/273] allow to randomize part type in MergeTree --- src/Storages/MergeTree/MergeTreeSettings.h | 5 ++- .../MergeTree/registerStorageMergeTree.cpp | 34 +++++++++++++++++++ 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 49847617d30..9adbc1a7b3d 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -20,7 +20,7 @@ struct Settings; M(UInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \ \ /** Data storing format settings. */ \ - M(UInt64, min_bytes_for_wide_part, 0, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ + M(UInt64, min_bytes_for_wide_part, 10485760, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ M(UInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \ M(UInt64, min_bytes_for_compact_part, 0, "Experimental. Minimal uncompressed size in bytes to create part in compact format instead of saving it in RAM", 0) \ M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ @@ -97,6 +97,9 @@ struct Settings; M(String, storage_policy, "default", "Name of storage disk policy", 0) \ M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \ \ + /** Settings for testing purposes */ \ + M(Bool, randomize_part_type, false, "For testing purposes only. Randomizes part type between wide and compact", 0) \ + \ /** Obsolete settings. Kept for backward compatibility only. */ \ M(UInt64, min_relative_delay_to_yield_leadership, 120, "Obsolete setting, does nothing.", 0) \ M(UInt64, check_delay_period, 60, "Obsolete setting, does nothing.", 0) \ diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index b0c422bd79f..5609c130aba 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -233,6 +234,25 @@ If you use the Replicated version of engines, see https://clickhouse.tech/docs/e } +static void randomizePartTypeSettings(const std::unique_ptr & storage_settings) +{ + static constexpr auto MAX_THRESHOLD_FOR_ROWS = 100000; + static constexpr auto MAX_THRESHOLD_FOR_BYTES = 1024 * 1024 * 10; + + /// Create all parts in wide format with probability 1/3. + if (thread_local_rng() % 3 == 0) + { + storage_settings->min_rows_for_wide_part = 0; + storage_settings->min_bytes_for_wide_part = 0; + } + else + { + storage_settings->min_rows_for_wide_part = std::uniform_int_distribution{0, MAX_THRESHOLD_FOR_ROWS}(thread_local_rng); + storage_settings->min_bytes_for_wide_part = std::uniform_int_distribution{0, MAX_THRESHOLD_FOR_BYTES}(thread_local_rng); + } +} + + static StoragePtr create(const StorageFactory::Arguments & args) { /** [Replicated][|Summing|Collapsing|Aggregating|Replacing|Graphite]MergeTree (2 * 7 combinations) engines @@ -652,6 +672,20 @@ static StoragePtr create(const StorageFactory::Arguments & args) ++arg_num; } + /// Allow to randomize part type for tests to cover more cases. + /// But if settings were set explicitly restrict it. + if (storage_settings->randomize_part_type + && !storage_settings->min_rows_for_wide_part.changed + && !storage_settings->min_bytes_for_wide_part.changed) + { + randomizePartTypeSettings(storage_settings); + LOG_INFO(&Poco::Logger::get(args.table_id.getNameForLogs() + " (registerStorageMergeTree)"), + "Applied setting 'randomize_part_type'. " + "Setting 'min_rows_for_wide_part' changed to {}. " + "Setting 'min_bytes_for_wide_part' changed to {}.", + storage_settings->min_rows_for_wide_part, storage_settings->min_bytes_for_wide_part); + } + if (arg_num != arg_cnt) throw Exception("Wrong number of engine arguments.", ErrorCodes::BAD_ARGUMENTS); From a173a863a8e971732524cceeb85de1dbf03d1b11 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Thu, 10 Sep 2020 11:03:35 +0800 Subject: [PATCH 060/273] fix style error --- src/Parsers/ASTShowTablesQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ASTShowTablesQuery.cpp b/src/Parsers/ASTShowTablesQuery.cpp index b59ba07d03e..1e8dad13ad3 100644 --- a/src/Parsers/ASTShowTablesQuery.cpp +++ b/src/Parsers/ASTShowTablesQuery.cpp @@ -63,7 +63,7 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(from); - formatLike(settings); + formatLike(settings); if (where_expression) { @@ -71,7 +71,7 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format where_expression->formatImpl(settings, state, frame); } - formatLimit(settings, state, frame); + formatLimit(settings, state, frame); } } From fea763bb751fd0fb4abfef9ff34acdabb8b8e0d8 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 10 Sep 2020 11:37:00 +0800 Subject: [PATCH 061/273] ISSUES-4006 support decimal 256 for mysql block input stream --- src/Core/ExternalResultDescription.cpp | 2 ++ src/Core/ExternalResultDescription.h | 3 ++- src/Formats/MySQLBlockInputStream.cpp | 3 ++- 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Core/ExternalResultDescription.cpp b/src/Core/ExternalResultDescription.cpp index 941ee003c94..7165d73b7d0 100644 --- a/src/Core/ExternalResultDescription.cpp +++ b/src/Core/ExternalResultDescription.cpp @@ -74,6 +74,8 @@ void ExternalResultDescription::init(const Block & sample_block_) types.emplace_back(ValueType::vtDecimal64, is_nullable); else if (typeid_cast *>(type)) types.emplace_back(ValueType::vtDecimal128, is_nullable); + else if (typeid_cast *>(type)) + types.emplace_back(ValueType::vtDecimal256, is_nullable); else throw Exception{"Unsupported type " + type->getName(), ErrorCodes::UNKNOWN_TYPE}; } diff --git a/src/Core/ExternalResultDescription.h b/src/Core/ExternalResultDescription.h index 29294fcf2c8..f8ba2a6bba2 100644 --- a/src/Core/ExternalResultDescription.h +++ b/src/Core/ExternalResultDescription.h @@ -29,7 +29,8 @@ struct ExternalResultDescription vtDateTime64, vtDecimal32, vtDecimal64, - vtDecimal128 + vtDecimal128, + vtDecimal256 }; Block sample_block; diff --git a/src/Formats/MySQLBlockInputStream.cpp b/src/Formats/MySQLBlockInputStream.cpp index f85680c0031..be1e254b22f 100644 --- a/src/Formats/MySQLBlockInputStream.cpp +++ b/src/Formats/MySQLBlockInputStream.cpp @@ -90,7 +90,8 @@ namespace case ValueType::vtDateTime64:[[fallthrough]]; case ValueType::vtDecimal32: [[fallthrough]]; case ValueType::vtDecimal64: [[fallthrough]]; - case ValueType::vtDecimal128: + case ValueType::vtDecimal128:[[fallthrough]]; + case ValueType::vtDecimal256: { ReadBuffer buffer(const_cast(value.data()), value.size(), 0); data_type.deserializeAsWholeText(column, buffer, FormatSettings{}); From 338b69201d126a9ccc0be828b6fed4f6a0affef9 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 10 Sep 2020 17:20:28 +0800 Subject: [PATCH 062/273] ISSUES-4006 add test decimal when dump stage --- .../materialize_with_ddl.py | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index fa31c43e6c1..a953202bff0 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -98,19 +98,29 @@ def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_nam def materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, mysql_node, service_name): mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) - mysql_node.query("CREATE TABLE test_database.test_table_1 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), " - "_decimal DECIMAL(65, 30)) ENGINE = InnoDB;") + mysql_node.query("CREATE TABLE test_database.test_table_1 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), _decimal DECIMAL(65, 30)) ENGINE = InnoDB;") mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(2, '2020-01-01 01:02:03.000000', '2020-01-01 01:02:03.000', ." + ('0' * 29) + "1)") mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(3, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.99', -" + ('9' * 35) + "." + ('9' * 30) + ")") mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") + + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", "1\t2020-01-01 01:02:03.999999\t2020-01-01 01:02:03.999\t" + ('9' * 35) + "." + ('9' * 30) + "\n" "2\t2020-01-01 01:02:03.000000\t2020-01-01 01:02:03.000\t0." + ('0' * 29) + "1\n" "3\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:03.990\t-" + ('9' * 35) + "." + ('9' * 30) + "\n" "4\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:04.000\t-0." + ('0' * 29) + "1\n") + + mysql_node.query("CREATE TABLE test_database.test_table_2 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), _decimal DECIMAL(65, 30)) ENGINE = InnoDB;") + mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") + mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(2, '2020-01-01 01:02:03.000000', '2020-01-01 01:02:03.000', ." + ('0' * 29) + "1)") + mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(3, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.99', -" + ('9' * 35) + "." + ('9' * 30) + ")") + mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") + check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY key FORMAT TSV", + "1\t2020-01-01 01:02:03.999999\t2020-01-01 01:02:03.999\t" + ('9' * 35) + "." + ('9' * 30) + "\n" + "2\t2020-01-01 01:02:03.000000\t2020-01-01 01:02:03.000\t0." + ('0' * 29) + "1\n" + "3\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:03.990\t-" + ('9' * 35) + "." + ('9' * 30) + "\n" + "4\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:04.000\t-0." + ('0' * 29) + "1\n") clickhouse_node.query("DROP DATABASE test_database") mysql_node.query("DROP DATABASE test_database") From 223fc4d1e7b0ffd6ad7f2b2226d2210eec5af64d Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 10 Sep 2020 12:59:10 +0300 Subject: [PATCH 063/273] Avoid error while building the report on broken perf tests --- docker/test/performance-comparison/compare.sh | 12 ++++++++++++ src/Interpreters/ExpressionActions.cpp | 12 ++++++++++-- 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 364e9994ab7..2ae7910dcaa 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -394,12 +394,24 @@ create table query_run_metrics_denorm engine File(TSV, 'analyze/query-run-metric order by test, query_index, metric_names, version, query_id ; +-- Filter out tests that don't have an even number of runs, to avoid breaking +-- the further calculations. This may happen if there was an error during the +-- test runs, e.g. the server died. It will be reported in test errors, so we +-- don't have to report it again. +create view broken_tests as + select test_name + from query_runs + group by test_name + having count(*) % 2 == 0 + ; + -- This is for statistical processing with eqmed.sql create table query_run_metrics_for_stats engine File( TSV, -- do not add header -- will parse with grep 'analyze/query-run-metrics-for-stats.tsv') as select test, query_index, 0 run, version, metric_values from query_run_metric_arrays + where test not in broken_tests order by test, query_index, run, version ; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 33fa6215160..0c287e4026d 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -607,8 +607,16 @@ void ExpressionActions::execute(Block & block, bool dry_run) const { for (const auto & action : actions) { - action.execute(block, dry_run); - checkLimits(block); + try + { + action.execute(block, dry_run); + checkLimits(block); + } + catch (Exception & e) + { + e.addMessage(fmt::format("while executing '{}'", action.toString())); + throw; + } } } From a2a647eb1caac92b13f73b04651c7d64b66c0fc1 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 10 Sep 2020 13:02:45 +0300 Subject: [PATCH 064/273] fixup --- docker/test/performance-comparison/compare.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 2ae7910dcaa..08f4cb599ab 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -398,11 +398,11 @@ create table query_run_metrics_denorm engine File(TSV, 'analyze/query-run-metric -- the further calculations. This may happen if there was an error during the -- test runs, e.g. the server died. It will be reported in test errors, so we -- don't have to report it again. -create view broken_tests as - select test_name +create view broken_queries as + select test, query_index from query_runs - group by test_name - having count(*) % 2 == 0 + group by test, query_index + having count(*) % 2 != 0 ; -- This is for statistical processing with eqmed.sql @@ -411,7 +411,7 @@ create table query_run_metrics_for_stats engine File( 'analyze/query-run-metrics-for-stats.tsv') as select test, query_index, 0 run, version, metric_values from query_run_metric_arrays - where test not in broken_tests + where (test, query_index) not in broken_queries order by test, query_index, run, version ; From 8689797efc4d749f4cda139818b1a94caccfe628 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 10 Sep 2020 19:57:26 +0300 Subject: [PATCH 065/273] 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 6031e6bae95f8207ab554a422244ecf160aa90d8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 10 Sep 2020 20:09:19 +0300 Subject: [PATCH 066/273] fix tests --- .../configs/wide_parts_only.xml | 6 +++++ .../test_adaptive_granularity/test.py | 24 ++++++++++++++----- .../configs/compact_parts.xml | 1 + .../test_polymorphic_parts/test.py | 1 + ...46_clear_column_in_partition_zookeeper.sql | 6 ++--- ...system_columns_and_system_tables.reference | 2 +- ...00753_system_columns_and_system_tables.sql | 3 ++- ...ndex_granularity_collapsing_merge_tree.sql | 2 +- ..._adaptive_index_granularity_merge_tree.sql | 12 ++-------- .../queries/0_stateless/00933_ttl_simple.sql | 1 - ...ms_in_system_parts_columns_table.reference | 2 +- ...hecksums_in_system_parts_columns_table.sql | 10 ++------ 12 files changed, 38 insertions(+), 32 deletions(-) create mode 100644 tests/integration/test_adaptive_granularity/configs/wide_parts_only.xml diff --git a/tests/integration/test_adaptive_granularity/configs/wide_parts_only.xml b/tests/integration/test_adaptive_granularity/configs/wide_parts_only.xml new file mode 100644 index 00000000000..42e2173f718 --- /dev/null +++ b/tests/integration/test_adaptive_granularity/configs/wide_parts_only.xml @@ -0,0 +1,6 @@ + + + 0 + 0 + + diff --git a/tests/integration/test_adaptive_granularity/test.py b/tests/integration/test_adaptive_granularity/test.py index 21d65588de4..7efafb4ddd1 100644 --- a/tests/integration/test_adaptive_granularity/test.py +++ b/tests/integration/test_adaptive_granularity/test.py @@ -13,10 +13,10 @@ node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml' node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) node3 = cluster.add_instance('node3', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server', tag='19.6.3.18', with_installed_binary=True) -node4 = cluster.add_instance('node4', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node4 = cluster.add_instance('node4', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/wide_parts_only.xml'], with_zookeeper=True) node5 = cluster.add_instance('node5', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server', tag='19.1.15', with_installed_binary=True) -node6 = cluster.add_instance('node6', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) +node6 = cluster.add_instance('node6', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/wide_parts_only.xml'], with_zookeeper=True) node7 = cluster.add_instance('node7', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server', tag='19.6.3.18', stay_alive=True, with_installed_binary=True) node8 = cluster.add_instance('node8', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server', tag='19.1.15', stay_alive=True, with_installed_binary=True) @@ -270,8 +270,14 @@ def test_mixed_granularity_single_node(start_dynamic_cluster, node): node.query("INSERT INTO table_with_default_granularity VALUES (toDate('2018-09-01'), 1, 333), (toDate('2018-09-02'), 2, 444)") def callback(n): - n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml", "1") - n.replace_config("/etc/clickhouse-server/config.d/merge_tree_settings.xml", "1") + new_config = """ + + 1 + 0 +""" + + n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml", new_config) + n.replace_config("/etc/clickhouse-server/config.d/merge_tree_settings.xml", new_config) node.restart_with_latest_version(callback_onstop=callback) node.query("SYSTEM RELOAD CONFIG") @@ -304,8 +310,14 @@ def test_version_update_two_nodes(start_dynamic_cluster): node12.query("SYSTEM SYNC REPLICA table_with_default_granularity", timeout=20) assert node12.query("SELECT COUNT() FROM table_with_default_granularity") == '2\n' def callback(n): - n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml", "0") - n.replace_config("/etc/clickhouse-server/config.d/merge_tree_settings.xml", "0") + new_config = """ + + 0 + 0 +""" + + n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml", new_config) + n.replace_config("/etc/clickhouse-server/config.d/merge_tree_settings.xml", new_config) node12.restart_with_latest_version(callback_onstop=callback) diff --git a/tests/integration/test_polymorphic_parts/configs/compact_parts.xml b/tests/integration/test_polymorphic_parts/configs/compact_parts.xml index e14c3f0ceae..5b3afe65d92 100644 --- a/tests/integration/test_polymorphic_parts/configs/compact_parts.xml +++ b/tests/integration/test_polymorphic_parts/configs/compact_parts.xml @@ -1,5 +1,6 @@ 512 + 0 diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index ed89f768d4c..cf2268bc831 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -42,6 +42,7 @@ def create_tables(name, nodes, node_settings, shard): ORDER BY id SETTINGS index_granularity = 64, index_granularity_bytes = {index_granularity_bytes}, min_rows_for_wide_part = {min_rows_for_wide_part}, min_rows_for_compact_part = {min_rows_for_compact_part}, + min_bytes_for_wide_part = 0, min_bytes_for_compact_part = 0, in_memory_parts_enable_wal = 1 '''.format(name=name, shard=shard, repl=i, **settings)) diff --git a/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql b/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql index 56f3a654682..e6de5a91ce3 100644 --- a/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql +++ b/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql @@ -3,7 +3,7 @@ SELECT '===Ordinary case==='; SET replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS clear_column; -CREATE TABLE clear_column (d Date, num Int64, str String) ENGINE = MergeTree(d, d, 8192); +CREATE TABLE clear_column (d Date, num Int64, str String) ENGINE = MergeTree ORDER BY d PARTITION by d SETTINGS min_bytes_for_wide_part = 0; INSERT INTO clear_column VALUES ('2016-12-12', 1, 'a'), ('2016-11-12', 2, 'b'); @@ -24,8 +24,8 @@ SELECT '===Replicated case==='; DROP TABLE IF EXISTS clear_column1; DROP TABLE IF EXISTS clear_column2; SELECT sleep(1) FORMAT Null; -CREATE TABLE clear_column1 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '1', d, d, 8192); -CREATE TABLE clear_column2 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '2', d, d, 8192); +CREATE TABLE clear_column1 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '1') ORDER BY d PARTITION by d SETTINGS min_bytes_for_wide_part = 0; +CREATE TABLE clear_column2 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '2') ORDER BY d PARTITION by d SETTINGS min_bytes_for_wide_part = 0; INSERT INTO clear_column1 (d) VALUES ('2000-01-01'), ('2000-02-01'); SYSTEM SYNC REPLICA clear_column2; diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference b/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference index 47c15da4b22..4d1fab83cc1 100644 --- a/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference +++ b/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference @@ -6,7 +6,7 @@ │ name2 │ 1 │ 0 │ 0 │ 0 │ │ name3 │ 0 │ 0 │ 0 │ 0 │ └───────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘ -147 1 +231 1 ┌─name────────────────┬─partition_key─┬─sorting_key───┬─primary_key─┬─sampling_key─┐ │ check_system_tables │ date │ date, version │ date │ │ └─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘ diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql b/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql index 1d7faa32952..9b9fa04e6b0 100644 --- a/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql +++ b/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql @@ -9,7 +9,8 @@ CREATE TABLE check_system_tables ) ENGINE = MergeTree() ORDER BY name1 PARTITION BY name2 - SAMPLE BY name1; + SAMPLE BY name1 + SETTINGS min_bytes_for_wide_part = 0; SELECT name, partition_key, sorting_key, primary_key, sampling_key, storage_policy, total_rows FROM system.tables diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql index 74159738bce..d4c19cbe8f2 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql @@ -41,7 +41,7 @@ CREATE TABLE four_rows_per_granule ( Sign Int8 ) ENGINE CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes=110, min_index_granularity_bytes=100, write_final_mark = 0, - + min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, vertical_merge_algorithm_min_columns_to_activate=0; diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.sql index f2e9e9749e3..249c6eebfcf 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.sql @@ -61,11 +61,7 @@ CREATE TABLE four_rows_per_granule ( k UInt64, v1 UInt64, v2 Int64 -<<<<<<< HEAD -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0, min_bytes_for_wide_part = 0; -======= -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 10, write_final_mark = 0; ->>>>>>> upstream/master +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 10, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO four_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -130,11 +126,7 @@ CREATE TABLE adaptive_granularity_alter ( k UInt64, v1 UInt64, v2 Int64 -<<<<<<< HEAD -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0, min_bytes_for_wide_part = 0; -======= -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0; ->>>>>>> upstream/master +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO adaptive_granularity_alter (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); diff --git a/tests/queries/0_stateless/00933_ttl_simple.sql b/tests/queries/0_stateless/00933_ttl_simple.sql index 83d9962043d..c0adcd21e62 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.sql +++ b/tests/queries/0_stateless/00933_ttl_simple.sql @@ -23,7 +23,6 @@ create table ttl_00933_1 (d DateTime, a Int ttl d + interval 1 DAY) engine = Mer insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 3); -optimize table ttl_00933_1 final; select * from ttl_00933_1 order by d; drop table if exists ttl_00933_1; diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference index 3bcfc00eded..099fe566817 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference @@ -1 +1 @@ -20000101_20000101_1_1_0 test_00961 b78f351b7498ecc9d4732ad29c3952de 1d4b7fbf05d0fc5c2f4559ca75aa32f7 38f047b57fd1bb81cf77e273deb34218 +20000101_1_1_0 test_00961 5f2e2d4bbc14336f44037e3ac667f247 ed226557cd4e18ecf3ae06c6d5e6725c da96ff1e527a8a1f908ddf2b1d0af239 diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql index 46daa0bf711..792bf62f9b1 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql @@ -1,21 +1,15 @@ DROP TABLE IF EXISTS test_00961; -CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree(d, (a, b), 111); +CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) + ENGINE = MergeTree PARTITION BY d ORDER BY (a, b) SETTINGS index_granularity = 111, min_bytes_for_wide_part = 0; INSERT INTO test_00961 VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789); SELECT -<<<<<<< HEAD name, table, hash_of_all_files, hash_of_uncompressed_files, -======= - name, - table, - hash_of_all_files, - hash_of_uncompressed_files, ->>>>>>> upstream/master uncompressed_hash_of_compressed_files FROM system.parts WHERE table = 'test_00961' and database = currentDatabase(); From a420976041581f4e0af3e9363a16a3c2aa119d97 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Thu, 10 Sep 2020 20:12:06 +0300 Subject: [PATCH 067/273] Bump CI [2]. From fa6d88b3b29f9a0e852e009651c32652e0201fad Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 10 Sep 2020 21:43:02 +0300 Subject: [PATCH 068/273] fix more tests --- .../test_filesystem_layout/test.py | 2 +- .../configs/config.d/storage_conf.xml | 4 ++++ tests/integration/test_partition/test.py | 2 +- .../configs/config.d/storage_conf.xml | 4 ++++ .../test_replicated_merge_tree_s3/test.py | 22 ++++++++++++++----- tests/integration/test_ttl_replicated/test.py | 10 +++++---- ...46_clear_column_in_partition_zookeeper.sql | 6 ++--- .../0_stateless/00933_ttl_simple.reference | 8 +++---- .../queries/0_stateless/00933_ttl_simple.sql | 15 ++++++++----- 9 files changed, 48 insertions(+), 25 deletions(-) diff --git a/tests/integration/test_filesystem_layout/test.py b/tests/integration/test_filesystem_layout/test.py index 83389b3d9bd..777e5ab7b9a 100644 --- a/tests/integration/test_filesystem_layout/test.py +++ b/tests/integration/test_filesystem_layout/test.py @@ -19,7 +19,7 @@ def test_file_path_escaping(started_cluster): node.query('CREATE DATABASE IF NOT EXISTS test ENGINE = Ordinary') node.query(''' CREATE TABLE test.`T.a_b,l-e!` (`~Id` UInt32) - ENGINE = MergeTree() PARTITION BY `~Id` ORDER BY `~Id`; + ENGINE = MergeTree() PARTITION BY `~Id` ORDER BY `~Id` SETTINGS min_bytes_for_wide_part = 0; ''') node.query('''INSERT INTO test.`T.a_b,l-e!` VALUES (1);''') node.query('''ALTER TABLE test.`T.a_b,l-e!` FREEZE;''') diff --git a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml index d097675ca63..343f248c5fb 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml @@ -25,4 +25,8 @@ + + + 0 + diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index 80fbe947316..2a2bbe205b5 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -178,7 +178,7 @@ def test_attach_check_all_parts(attach_check_all_parts_table): exec_bash('cp -pr {} {}'.format(path_to_detached + '0_3_3_0', path_to_detached + 'deleting_0_7_7_0')) error = instance.client.query_and_get_error("ALTER TABLE test.attach_partition ATTACH PARTITION 0") - assert 0 <= error.find('No columns in part 0_5_5_0') + assert 0 <= error.find('No columns in part 0_5_5_0') or 0 <= error.find('No columns.txt in part 0_5_5_0') parts = q("SElECT name FROM system.parts WHERE table='attach_partition' AND database='test' ORDER BY name") assert TSV(parts) == TSV('1_2_2_0\n1_4_4_0') diff --git a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml index b32770095fc..f3b7f959ce9 100644 --- a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml @@ -18,4 +18,8 @@ + + + 0 + diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index a77a69b842b..612b50becf7 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -30,7 +30,8 @@ def cluster(): FILES_OVERHEAD = 1 FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files -FILES_OVERHEAD_PER_PART = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 +FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 +FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 def random_string(length): @@ -44,7 +45,7 @@ def generate_values(date_str, count, sign=1): return ",".join(["('{}',{},'{}')".format(x, y, z) for x, y, z in data]) -def create_table(cluster): +def create_table(cluster, additional_settings=None): create_table_statement = """ CREATE TABLE s3_test ( dt Date, @@ -56,6 +57,9 @@ def create_table(cluster): ORDER BY (dt, id) SETTINGS storage_policy='s3' """ + if additional_settings: + create_table_statement += "," + create_table_statement += additional_settings for node in cluster.instances.values(): node.query(create_table_statement) @@ -72,9 +76,15 @@ def drop_table(cluster): for obj in list(minio.list_objects(cluster.minio_bucket, 'data/')): minio.remove_object(cluster.minio_bucket, obj.object_name) - -def test_insert_select_replicated(cluster): - create_table(cluster) +@pytest.mark.parametrize( + "min_rows_for_wide_part,files_per_part", + [ + (0, FILES_OVERHEAD_PER_PART_WIDE), + (8192, FILES_OVERHEAD_PER_PART_COMPACT) + ] +) +def test_insert_select_replicated(cluster, min_rows_for_wide_part, files_per_part): + create_table(cluster, additional_settings="min_rows_for_wide_part={}".format(min_rows_for_wide_part)) all_values = "" for node_idx in range(1, 4): @@ -90,4 +100,4 @@ def test_insert_select_replicated(cluster): assert node.query("SELECT * FROM s3_test order by dt, id FORMAT Values", settings={"select_sequential_consistency": 1}) == all_values minio = cluster.minio_client - assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 3 * (FILES_OVERHEAD + FILES_OVERHEAD_PER_PART * 3) + assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 3 * (FILES_OVERHEAD + files_per_part * 3) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index 0f201f569b3..39d595662d0 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -30,7 +30,7 @@ def drop_table(nodes, table_name): node.query("DROP TABLE IF EXISTS {} NO DELAY".format(table_name)) time.sleep(1) - +# Column TTL works only with wide parts, because it's very expensive to apply it for compact parts def test_ttl_columns(started_cluster): drop_table([node1, node2], "test_ttl") for node in [node1, node2]: @@ -38,7 +38,7 @@ def test_ttl_columns(started_cluster): ''' CREATE TABLE test_ttl(date DateTime, id UInt32, a Int32 TTL date + INTERVAL 1 DAY, b Int32 TTL date + INTERVAL 1 MONTH) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl', '{replica}') - ORDER BY id PARTITION BY toDayOfMonth(date) SETTINGS merge_with_ttl_timeout=0; + ORDER BY id PARTITION BY toDayOfMonth(date) SETTINGS merge_with_ttl_timeout=0, min_bytes_for_wide_part=0; '''.format(replica=node.name)) node1.query("INSERT INTO test_ttl VALUES (toDateTime('2000-10-10 00:00:00'), 1, 1, 3)") @@ -59,7 +59,8 @@ def test_merge_with_ttl_timeout(started_cluster): ''' CREATE TABLE {table}(date DateTime, id UInt32, a Int32 TTL date + INTERVAL 1 DAY, b Int32 TTL date + INTERVAL 1 MONTH) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{table}', '{replica}') - ORDER BY id PARTITION BY toDayOfMonth(date); + ORDER BY id PARTITION BY toDayOfMonth(date) + SETTINGS min_bytes_for_wide_part=0; '''.format(replica=node.name, table=table)) node1.query("SYSTEM STOP TTL MERGES {table}".format(table=table)) @@ -198,7 +199,7 @@ def test_ttl_double_delete_rule_returns_error(started_cluster): CREATE TABLE test_ttl(date DateTime, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) - TTL date + INTERVAL 1 DAY, date + INTERVAL 2 DAY SETTINGS merge_with_ttl_timeout=0; + TTL date + INTERVAL 1 DAY, date + INTERVAL 2 DAY SETTINGS merge_with_ttl_timeout=0 '''.format(replica=node1.name)) assert False except client.QueryRuntimeException: @@ -246,6 +247,7 @@ limitations under the License.""" ) ENGINE = {engine} ORDER BY tuple() TTL d1 + INTERVAL 1 DAY DELETE + SETTINGS min_bytes_for_wide_part=0 """.format(name=name, engine=engine)) node1.query("""ALTER TABLE {name} MODIFY COLUMN s1 String TTL d1 + INTERVAL 1 SECOND""".format(name=name)) diff --git a/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql b/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql index e6de5a91ce3..bd6c12ffce4 100644 --- a/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql +++ b/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql @@ -3,7 +3,7 @@ SELECT '===Ordinary case==='; SET replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS clear_column; -CREATE TABLE clear_column (d Date, num Int64, str String) ENGINE = MergeTree ORDER BY d PARTITION by d SETTINGS min_bytes_for_wide_part = 0; +CREATE TABLE clear_column (d Date, num Int64, str String) ENGINE = MergeTree ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0; INSERT INTO clear_column VALUES ('2016-12-12', 1, 'a'), ('2016-11-12', 2, 'b'); @@ -24,8 +24,8 @@ SELECT '===Replicated case==='; DROP TABLE IF EXISTS clear_column1; DROP TABLE IF EXISTS clear_column2; SELECT sleep(1) FORMAT Null; -CREATE TABLE clear_column1 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '1') ORDER BY d PARTITION by d SETTINGS min_bytes_for_wide_part = 0; -CREATE TABLE clear_column2 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '2') ORDER BY d PARTITION by d SETTINGS min_bytes_for_wide_part = 0; +CREATE TABLE clear_column1 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '1') ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0; +CREATE TABLE clear_column2 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '2') ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0; INSERT INTO clear_column1 (d) VALUES ('2000-01-01'), ('2000-02-01'); SYSTEM SYNC REPLICA clear_column2; diff --git a/tests/queries/0_stateless/00933_ttl_simple.reference b/tests/queries/0_stateless/00933_ttl_simple.reference index a4ef8033328..e3982814eab 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.reference +++ b/tests/queries/0_stateless/00933_ttl_simple.reference @@ -6,11 +6,11 @@ 2000-10-10 00:00:00 0 2100-10-10 00:00:00 3 2100-10-10 2 -CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL now() - 1000\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL now() - 1000\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 1 0 -CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL now() + 1000\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL now() + 1000\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 1 1 -CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL today() - 1\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL today() - 1\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 1 0 -CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL today() + 1\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL today() + 1\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 1 1 diff --git a/tests/queries/0_stateless/00933_ttl_simple.sql b/tests/queries/0_stateless/00933_ttl_simple.sql index c0adcd21e62..aa8b33b2999 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.sql +++ b/tests/queries/0_stateless/00933_ttl_simple.sql @@ -1,6 +1,8 @@ drop table if exists ttl_00933_1; -create table ttl_00933_1 (d DateTime, a Int ttl d + interval 1 second, b Int ttl d + interval 1 second) engine = MergeTree order by tuple() partition by toMinute(d); +-- Column TTL works only with wide parts, because it's very expensive to apply it for compact parts + +create table ttl_00933_1 (d DateTime, a Int ttl d + interval 1 second, b Int ttl d + interval 1 second) engine = MergeTree order by tuple() partition by toMinute(d) settings min_bytes_for_wide_part = 0; insert into ttl_00933_1 values (now(), 1, 2); insert into ttl_00933_1 values (now(), 3, 4); select sleep(1.1) format Null; @@ -19,10 +21,11 @@ select a, b from ttl_00933_1; drop table if exists ttl_00933_1; -create table ttl_00933_1 (d DateTime, a Int ttl d + interval 1 DAY) engine = MergeTree order by tuple() partition by toDayOfMonth(d); +create table ttl_00933_1 (d DateTime, a Int ttl d + interval 1 DAY) engine = MergeTree order by tuple() partition by toDayOfMonth(d) settings min_bytes_for_wide_part = 0; insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 3); +optimize table ttl_00933_1 final; select * from ttl_00933_1 order by d; drop table if exists ttl_00933_1; @@ -44,7 +47,7 @@ select * from ttl_00933_1 order by d; -- const DateTime TTL positive drop table if exists ttl_00933_1; -create table ttl_00933_1 (b Int, a Int ttl now()-1000) engine = MergeTree order by tuple() partition by tuple(); +create table ttl_00933_1 (b Int, a Int ttl now()-1000) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); optimize table ttl_00933_1 final; @@ -52,7 +55,7 @@ select * from ttl_00933_1; -- const DateTime TTL negative drop table if exists ttl_00933_1; -create table ttl_00933_1 (b Int, a Int ttl now()+1000) engine = MergeTree order by tuple() partition by tuple(); +create table ttl_00933_1 (b Int, a Int ttl now()+1000) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); optimize table ttl_00933_1 final; @@ -60,7 +63,7 @@ select * from ttl_00933_1; -- const Date TTL positive drop table if exists ttl_00933_1; -create table ttl_00933_1 (b Int, a Int ttl today()-1) engine = MergeTree order by tuple() partition by tuple(); +create table ttl_00933_1 (b Int, a Int ttl today()-1) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); optimize table ttl_00933_1 final; @@ -68,7 +71,7 @@ select * from ttl_00933_1; -- const Date TTL negative drop table if exists ttl_00933_1; -create table ttl_00933_1 (b Int, a Int ttl today()+1) engine = MergeTree order by tuple() partition by tuple(); +create table ttl_00933_1 (b Int, a Int ttl today()+1) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); optimize table ttl_00933_1 final; From 8a201a28c04e06f1a2ebb03d51c0e1d8e983680a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 11 Sep 2020 03:14:35 +0300 Subject: [PATCH 069/273] remove skip list for tests with polymorphic parts --- tests/clickhouse-test | 9 --------- tests/queries/skip_list.json | 27 --------------------------- 2 files changed, 36 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 6bfad37d8ad..a3bed189d55 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -506,15 +506,6 @@ def collect_build_flags(client): else: raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) - clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - (stdout, stderr) = clickhouse_proc.communicate("SELECT value FROM system.merge_tree_settings WHERE name = 'min_bytes_for_wide_part'") - - if clickhouse_proc.returncode == 0: - if '10485760' in stdout: - result.append(BuildFlags.POLYMORPHIC_PARTS) - else: - raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) - return result diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index adfc5f0e582..535f2757e43 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -127,32 +127,5 @@ "01130_in_memory_parts_partitons", "01225_show_create_table_from_dictionary", "01224_no_superfluous_dict_reload" - ], - "polymorphic-parts": [ - /// These tests fail with compact parts, because they - /// check some implementation defined things - /// like checksums, computed granularity, ProfileEvents, etc. - "avx", - "01045_order_by_pk_special_storages", - "01042_check_query_and_last_granule_size", - "00961_checksums_in_system_parts_columns_table", - "00933_test_fix_extra_seek_on_compressed_cache", - "00926_adaptive_index_granularity_collapsing_merge_tree", - "00926_adaptive_index_granularity_merge_tree", - "00926_adaptive_index_granularity_replacing_merge_tree", - "00926_adaptive_index_granularity_versioned_collapsing_merge_tree", - "00804_test_delta_codec_compression", - "00731_long_merge_tree_select_opened_files", - "00653_verification_monotonic_data_load", - "00484_preferred_max_column_in_block_size_bytes", - "00446_clear_column_in_partition_zookeeper", - "00443_preferred_block_size_bytes", - "00160_merge_and_index_in_in", - "01055_compact_parts", - "01039_mergetree_exec_time", - "00933_ttl_simple", /// Maybe it's worth to fix it - "00753_system_columns_and_system_tables", - "01343_min_bytes_to_use_mmap_io", - "01344_min_bytes_to_use_mmap_io_index" ] } From 6047df2c37c6afea1afabd08cc4406c36f9a62e0 Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Fri, 11 Sep 2020 06:18:36 +0000 Subject: [PATCH 070/273] Bump numpy from 1.19.1 to 1.19.2 in /docs/tools Bumps [numpy](https://github.com/numpy/numpy) from 1.19.1 to 1.19.2. - [Release notes](https://github.com/numpy/numpy/releases) - [Changelog](https://github.com/numpy/numpy/blob/master/doc/HOWTO_RELEASE.rst.txt) - [Commits](https://github.com/numpy/numpy/compare/v1.19.1...v1.19.2) Signed-off-by: dependabot-preview[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 9e916489ea4..a3949892829 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -22,7 +22,7 @@ mkdocs-macros-plugin==0.4.9 nltk==3.5 nose==1.3.7 protobuf==3.13.0 -numpy==1.19.1 +numpy==1.19.2 Pygments==2.5.2 pymdown-extensions==8.0 python-slugify==4.0.1 From 7ff7ee6aac2d234625780eeb98e4cfbb7b88c5f8 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 11 Sep 2020 15:37:14 +0800 Subject: [PATCH 071/273] ISSUES-4006 try fix integration test --- .../test_materialize_mysql_database/materialize_with_ddl.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index a953202bff0..869c2e88c96 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -105,6 +105,8 @@ def materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, mysql_ mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", "1\t2020-01-01 01:02:03.999999\t2020-01-01 01:02:03.999\t" + ('9' * 35) + "." + ('9' * 30) + "\n" "2\t2020-01-01 01:02:03.000000\t2020-01-01 01:02:03.000\t0." + ('0' * 29) + "1\n" From dd867b787f0de6d6d7dca46a6bcf451990ceed6d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 3 Sep 2020 01:35:47 +0300 Subject: [PATCH 072/273] Allow parallel execution of distributed DDL Add distributed_ddl.pool_size to control maximum parallel to handle distributed DDL. Also: - convert Exception constructors to fmt-like - use sleepFor* over std::this_thread::sleep_for() --- programs/server/Server.cpp | 5 +- programs/server/config.xml | 3 + src/Interpreters/DDLWorker.cpp | 259 +++++++++++++++++---------------- src/Interpreters/DDLWorker.h | 31 ++-- 4 files changed, 162 insertions(+), 136 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f24ba444203..e4fd351f091 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -708,7 +708,10 @@ int Server::main(const std::vector & /*args*/) { /// DDL worker should be started after all tables were loaded String ddl_zookeeper_path = config().getString("distributed_ddl.path", "/clickhouse/task_queue/ddl/"); - global_context->setDDLWorker(std::make_unique(ddl_zookeeper_path, *global_context, &config(), "distributed_ddl")); + int pool_size = config().getInt("distributed_ddl.pool_size", 1); + if (pool_size < 1) + throw Exception("distributed_ddl.pool_size should be greater then 0", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + global_context->setDDLWorker(std::make_unique(pool_size, ddl_zookeeper_path, *global_context, &config(), "distributed_ddl")); } std::unique_ptr dns_cache_updater; diff --git a/programs/server/config.xml b/programs/server/config.xml index af01e880dc2..d13978f9ee8 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -615,6 +615,9 @@ + + + diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index b9b52e2f3fe..526f15d921f 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -22,7 +22,6 @@ #include #include #include -#include #include #include #include @@ -38,10 +37,11 @@ #include #include #include +#include #include +#include #include #include -#include namespace DB @@ -144,7 +144,7 @@ struct DDLLogEntry rb >> "version: " >> version >> "\n"; if (version != CURRENT_VERSION) - throw Exception("Unknown DDLLogEntry format version: " + DB::toString(version), ErrorCodes::UNKNOWN_FORMAT_VERSION); + throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unknown DDLLogEntry format version: {}", version); Strings host_id_strings; rb >> "query: " >> escape >> query >> "\n"; @@ -308,9 +308,14 @@ static bool isSupportedAlterType(int type) } -DDLWorker::DDLWorker(const std::string & zk_root_dir, Context & context_, const Poco::Util::AbstractConfiguration * config, const String & prefix) - : context(context_), log(&Poco::Logger::get("DDLWorker")) +DDLWorker::DDLWorker(int pool_size_, const std::string & zk_root_dir, Context & context_, const Poco::Util::AbstractConfiguration * config, const String & prefix) + : context(context_) + , log(&Poco::Logger::get("DDLWorker")) + , pool_size(pool_size_) + , worker_pool(pool_size_) { + last_tasks.reserve(pool_size); + queue_dir = zk_root_dir; if (queue_dir.back() == '/') queue_dir.resize(queue_dir.size() - 1); @@ -343,6 +348,7 @@ DDLWorker::~DDLWorker() stop_flag = true; queue_updated_event->set(); cleanup_event->set(); + worker_pool.wait(); main_thread.join(); cleanup_thread.join(); } @@ -364,8 +370,27 @@ DDLWorker::ZooKeeperPtr DDLWorker::getAndSetZooKeeper() return current_zookeeper; } +void DDLWorker::recoverZooKeeper() +{ + LOG_DEBUG(log, "Recovering ZooKeeper session after: {}", getCurrentExceptionMessage(false)); -bool DDLWorker::initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper) + while (!stop_flag) + { + try + { + getAndSetZooKeeper(); + break; + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + sleepForSeconds(5); + } + } +} + + +DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper) { String node_data; String entry_path = queue_dir + "/" + entry_name; @@ -374,7 +399,7 @@ bool DDLWorker::initAndCheckTask(const String & entry_name, String & out_reason, { /// It is Ok that node could be deleted just now. It means that there are no current host in node's host list. out_reason = "The task was deleted"; - return false; + return {}; } auto task = std::make_unique(); @@ -405,7 +430,7 @@ bool DDLWorker::initAndCheckTask(const String & entry_name, String & out_reason, } out_reason = "Incorrect task format"; - return false; + return {}; } bool host_in_hostlist = false; @@ -433,12 +458,13 @@ bool DDLWorker::initAndCheckTask(const String & entry_name, String & out_reason, } } - if (host_in_hostlist) - current_task = std::move(task); - else + if (!host_in_hostlist) + { out_reason = "There is no a local address in host list"; + return {}; + } - return host_in_hostlist; + return task; } @@ -448,10 +474,9 @@ static void filterAndSortQueueNodes(Strings & all_nodes) std::sort(all_nodes.begin(), all_nodes.end()); } - -void DDLWorker::processTasks() +void DDLWorker::scheduleTasks() { - LOG_DEBUG(log, "Processing tasks"); + LOG_DEBUG(log, "Scheduling tasks"); auto zookeeper = tryGetZooKeeper(); Strings queue_nodes = zookeeper->getChildren(queue_dir, nullptr, queue_updated_event); @@ -459,86 +484,60 @@ void DDLWorker::processTasks() if (queue_nodes.empty()) return; - bool server_startup = last_processed_task_name.empty(); + bool server_startup = last_tasks.empty(); auto begin_node = server_startup ? queue_nodes.begin() - : std::upper_bound(queue_nodes.begin(), queue_nodes.end(), last_processed_task_name); + : std::upper_bound(queue_nodes.begin(), queue_nodes.end(), last_tasks.back()); for (auto it = begin_node; it != queue_nodes.end(); ++it) { String entry_name = *it; - if (current_task) + String reason; + auto task = initAndCheckTask(entry_name, reason, zookeeper); + if (!task) { - if (current_task->entry_name == entry_name) - { - LOG_INFO(log, "Trying to process task {} again", entry_name); - } - else - { - LOG_INFO(log, "Task {} was deleted from ZooKeeper before current host committed it", current_task->entry_name); - current_task = nullptr; - } + LOG_DEBUG(log, "Will not execute task {}: {}", entry_name, reason); + saveTask(entry_name); + continue; } - if (!current_task) + bool already_processed = zookeeper->exists(task->entry_path + "/finished/" + task->host_id_str); + if (!server_startup && !task->was_executed && already_processed) { - String reason; - if (!initAndCheckTask(entry_name, reason, zookeeper)) - { - LOG_DEBUG(log, "Will not execute task {}: {}", entry_name, reason); - last_processed_task_name = entry_name; - continue; - } - } - - DDLTask & task = *current_task; - - bool already_processed = zookeeper->exists(task.entry_path + "/finished/" + task.host_id_str); - if (!server_startup && !task.was_executed && already_processed) - { - throw Exception( - "Server expects that DDL task " + task.entry_name + " should be processed, but it was already processed according to ZK", - ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Server expects that DDL task {} should be processed, but it was already processed according to ZK", + entry_name); } if (!already_processed) { - try + worker_pool.scheduleOrThrowOnError([this, task_ptr = task.release()]() { - processTask(task, zookeeper); - } - catch (const Coordination::Exception & e) - { - if (server_startup && e.code == Coordination::Error::ZNONODE) - { - LOG_WARNING(log, "ZooKeeper NONODE error during startup. Ignoring entry {} ({}) : {}", task.entry_name, task.entry.query, getCurrentExceptionMessage(true)); - } - else - { - throw; - } - } - catch (...) - { - LOG_WARNING(log, "An error occurred while processing task {} ({}) : {}", task.entry_name, task.entry.query, getCurrentExceptionMessage(true)); - throw; - } + enqueueTask(DDLTaskPtr(task_ptr)); + }); } else { - LOG_DEBUG(log, "Task {} ({}) has been already processed", task.entry_name, task.entry.query); + LOG_DEBUG(log, "Task {} ({}) has been already processed", entry_name, task->entry.query); } - last_processed_task_name = task.entry_name; - current_task.reset(); + saveTask(entry_name); if (stop_flag) break; } } +void DDLWorker::saveTask(const String & entry_name) +{ + if (last_tasks.size() == pool_size) + { + last_tasks.erase(last_tasks.begin()); + } + last_tasks.emplace_back(entry_name); +} /// Parses query and resolves cluster and host in cluster void DDLWorker::parseQueryAndResolveHost(DDLTask & task) @@ -559,10 +558,9 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task) task.cluster_name = task.query_on_cluster->cluster; task.cluster = context.tryGetCluster(task.cluster_name); if (!task.cluster) - { - throw Exception("DDL task " + task.entry_name + " contains current host " + task.host_id.readableString() - + " in cluster " + task.cluster_name + ", but there are no such cluster here.", ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); - } + throw Exception(ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION, + "DDL task {} contains current host {} in cluster {}, but there are no such cluster here.", + task.entry_name, task.host_id.readableString(), task.cluster_name); /// Try to find host from task host list in cluster /// At the first, try find exact match (host name and ports should be literally equal) @@ -583,10 +581,9 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task) { if (default_database == address.default_database) { - throw Exception( - "There are two exactly the same ClickHouse instances " + address.readableString() + " in cluster " - + task.cluster_name, - ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); + throw Exception(ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION, + "There are two exactly the same ClickHouse instances {} in cluster {}", + address.readableString(), task.cluster_name); } else { @@ -600,9 +597,8 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task) auto * query_with_table = dynamic_cast(task.query.get()); if (!query_with_table || query_with_table->database.empty()) { - throw Exception( - "For a distributed DDL on circular replicated cluster its table name must be qualified by database name.", - ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); + throw Exception(ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION, + "For a distributed DDL on circular replicated cluster its table name must be qualified by database name."); } if (default_database == query_with_table->database) return; @@ -635,8 +631,9 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task) { if (found_via_resolving) { - throw Exception("There are two the same ClickHouse instances in cluster " + task.cluster_name + " : " - + task.address_in_cluster.readableString() + " and " + address.readableString(), ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); + throw Exception(ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION, + "There are two the same ClickHouse instances in cluster {} : {} and {}", + task.cluster_name, task.address_in_cluster.readableString(), address.readableString()); } else { @@ -651,8 +648,9 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task) if (!found_via_resolving) { - throw Exception("Not found host " + task.host_id.readableString() + " in definition of cluster " + task.cluster_name, - ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); + throw Exception(ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION, + "Not found host {} in definition of cluster {}", + task.host_id.readableString(), task.cluster_name); } else { @@ -673,7 +671,7 @@ bool DDLWorker::tryExecuteQuery(const String & query, const DDLTask & task, Exec try { - current_context = std::make_unique(context); + auto current_context = std::make_unique(context); current_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; current_context->setCurrentQueryId(""); // generate random query_id executeQuery(istr, ostr, false, *current_context, {}); @@ -707,8 +705,44 @@ void DDLWorker::attachToThreadGroup() } -void DDLWorker::processTask(DDLTask & task, const ZooKeeperPtr & zookeeper) +void DDLWorker::enqueueTask(DDLTaskPtr task_ptr) { + auto & task = *task_ptr; + + while (!stop_flag) + { + try + { + processTask(task); + return; + } + catch (const Coordination::Exception & e) + { + if (Coordination::isHardwareError(e.code)) + { + recoverZooKeeper(); + } + else if (e.code == Coordination::Error::ZNONODE) + { + LOG_ERROR(log, "ZooKeeper error: {}", getCurrentExceptionMessage(true)); + // TODO: retry? + } + else + { + LOG_ERROR(log, "Unexpected ZooKeeper error: {}.", getCurrentExceptionMessage(true)); + return; + } + } + catch (...) + { + LOG_WARNING(log, "An error occurred while processing task {} ({}) : {}", task.entry_name, task.entry.query, getCurrentExceptionMessage(true)); + } + } +} +void DDLWorker::processTask(DDLTask & task) +{ + auto zookeeper = tryGetZooKeeper(); + LOG_DEBUG(log, "Processing task {} ({})", task.entry_name, task.entry.query); String dummy; @@ -816,16 +850,17 @@ void DDLWorker::checkShardConfig(const String & table, const DDLTask & task, Sto if (storage->supportsReplication() && !config_is_replicated_shard) { - throw Exception("Table " + backQuote(table) + " is replicated, but shard #" + toString(task.host_shard_num + 1) + - " isn't replicated according to its cluster definition." - " Possibly true is forgotten in the cluster config.", - ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); + throw Exception(ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION, + "Table {} is replicated, but shard #{} isn't replicated according to its cluster definition. " + "Possibly true is forgotten in the cluster config.", + backQuote(table), task.host_shard_num + 1); } if (!storage->supportsReplication() && config_is_replicated_shard) { - throw Exception("Table " + backQuote(table) + " isn't replicated, but shard #" + toString(task.host_shard_num + 1) + - " is replicated according to its cluster definition", ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); + throw Exception(ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION, + "Table {} isn't replicated, but shard #{} is replicated according to its cluster definition", + backQuote(table), task.host_shard_num + 1); } } @@ -841,7 +876,7 @@ bool DDLWorker::tryExecuteQueryOnLeaderReplica( /// If we will develop new replicated storage if (!replicated_storage) - throw Exception("Storage type '" + storage->getName() + "' is not supported by distributed DDL", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Storage type '{}' is not supported by distributed DDL", storage->getName()); /// Generate unique name for shard node, it will be used to execute the query by only single host /// Shard node name has format 'replica_name1,replica_name2,...,replica_nameN' @@ -1118,7 +1153,7 @@ void DDLWorker::runMainThread() attachToThreadGroup(); cleanup_event->set(); - processTasks(); + scheduleTasks(); LOG_DEBUG(log, "Waiting a watch"); queue_updated_event->wait(); @@ -1127,23 +1162,7 @@ void DDLWorker::runMainThread() { if (Coordination::isHardwareError(e.code)) { - LOG_DEBUG(log, "Recovering ZooKeeper session after: {}", getCurrentExceptionMessage(false)); - - while (!stop_flag) - { - try - { - getAndSetZooKeeper(); - break; - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - - using namespace std::chrono_literals; - std::this_thread::sleep_for(5s); - } - } + recoverZooKeeper(); } else if (e.code == Coordination::Error::ZNONODE) { @@ -1260,28 +1279,24 @@ public: size_t num_unfinished_hosts = waiting_hosts.size() - num_hosts_finished; size_t num_active_hosts = current_active_hosts.size(); - std::stringstream msg; - msg << "Watching task " << node_path << " is executing longer than distributed_ddl_task_timeout" - << " (=" << timeout_seconds << ") seconds." - << " There are " << num_unfinished_hosts << " unfinished hosts" - << " (" << num_active_hosts << " of them are currently active)" - << ", they are going to execute the query in background"; - throw Exception(msg.str(), ErrorCodes::TIMEOUT_EXCEEDED); + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, + "Watching task {} is executing longer than distributed_ddl_task_timeout (={}) seconds. " + "There are {} unfinished hosts ({} of them are currently active), they are going to execute the query in background", + node_path, timeout_seconds, num_unfinished_hosts, num_active_hosts); } if (num_hosts_finished != 0 || try_number != 0) { - auto current_sleep_for = std::chrono::milliseconds(std::min(static_cast(1000), 50 * (try_number + 1))); - std::this_thread::sleep_for(current_sleep_for); + sleepForMilliseconds(std::min(1000, 50 * (try_number + 1))); } /// TODO: add shared lock if (!zookeeper->exists(node_path)) { - throw Exception("Cannot provide query execution status. The query's node " + node_path - + " has been deleted by the cleaner since it was finished (or its lifetime is expired)", - ErrorCodes::UNFINISHED); + throw Exception(ErrorCodes::UNFINISHED, + "Cannot provide query execution status. The query's node {} has been deleted by the cleaner since it was finished (or its lifetime is expired)", + node_path); } Strings new_hosts = getNewAndUpdate(getChildrenAllowNoNode(zookeeper, node_path + "/finished")); @@ -1304,7 +1319,7 @@ public: auto [host, port] = Cluster::Address::fromString(host_id); if (status.code != 0 && first_exception == nullptr) - first_exception = std::make_unique("There was an error on [" + host + ":" + toString(port) + "]: " + status.message, status.code); + first_exception = std::make_unique(status.code, "There was an error on [{}:{}]: {}", host, port, status.message); ++num_hosts_finished; diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index 544fb3da27d..f6b4dd00684 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -26,6 +26,7 @@ class ASTAlterQuery; class AccessRightsElements; struct DDLLogEntry; struct DDLTask; +using DDLTaskPtr = std::unique_ptr; /// Pushes distributed DDL query to the queue @@ -37,7 +38,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & conte class DDLWorker { public: - DDLWorker(const std::string & zk_root_dir, Context & context_, const Poco::Util::AbstractConfiguration * config, const String & prefix); + DDLWorker(int pool_size_, const std::string & zk_root_dir, Context & context_, const Poco::Util::AbstractConfiguration * config, const String & prefix); ~DDLWorker(); /// Pushes query into DDL queue, returns path to created node @@ -57,14 +58,19 @@ private: ZooKeeperPtr tryGetZooKeeper() const; /// If necessary, creates a new session and caches it. ZooKeeperPtr getAndSetZooKeeper(); + /// ZooKeeper recover loop (while not stopped). + void recoverZooKeeper(); - void processTasks(); + void checkCurrentTasks(); + void scheduleTasks(); + void saveTask(const String & entry_name); /// Reads entry and check that the host belongs to host list of the task - /// Returns true and sets current_task if entry parsed and the check is passed - bool initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper); + /// Returns non-empty DDLTaskPtr if entry parsed and the check is passed + DDLTaskPtr initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper); - void processTask(DDLTask & task, const ZooKeeperPtr & zookeeper); + void enqueueTask(DDLTaskPtr task); + void processTask(DDLTask & task); /// Check that query should be executed on leader replica only static bool taskShouldBeExecutedOnLeader(const ASTPtr ast_ddl, StoragePtr storage); @@ -101,32 +107,31 @@ private: void attachToThreadGroup(); private: - bool is_circular_replicated; + std::atomic is_circular_replicated = false; Context & context; Poco::Logger * log; - std::unique_ptr current_context; std::string host_fqdn; /// current host domain name std::string host_fqdn_id; /// host_name:port std::string queue_dir; /// dir with queue of queries - /// Name of last task that was skipped or successfully executed - std::string last_processed_task_name; - mutable std::mutex zookeeper_mutex; ZooKeeperPtr current_zookeeper; /// Save state of executed task to avoid duplicate execution on ZK error - using DDLTaskPtr = std::unique_ptr; - DDLTaskPtr current_task; + std::vector last_tasks; std::shared_ptr queue_updated_event = std::make_shared(); std::shared_ptr cleanup_event = std::make_shared(); - std::atomic stop_flag{false}; + std::atomic stop_flag = false; ThreadFromGlobalPool main_thread; ThreadFromGlobalPool cleanup_thread; + /// Size of the pool for query execution. + size_t pool_size = 1; + ThreadPool worker_pool; + /// Cleaning starts after new node event is received if the last cleaning wasn't made sooner than N seconds ago Int64 cleanup_delay_period = 60; // minute (in seconds) /// Delete node if its age is greater than that From 9c7f3a9a742fb9b96c176b22b85f4d0a9e8a306c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 12 Sep 2020 02:33:17 +0300 Subject: [PATCH 073/273] Add test_distributed_ddl_parallel --- .../test_distributed_ddl_parallel/__init__.py | 0 .../configs/ddl.xml | 5 ++ .../configs/dict.xml | 26 ++++++ .../configs/remote_servers.xml | 18 ++++ .../test_distributed_ddl_parallel/test.py | 89 +++++++++++++++++++ 5 files changed, 138 insertions(+) create mode 100644 tests/integration/test_distributed_ddl_parallel/__init__.py create mode 100644 tests/integration/test_distributed_ddl_parallel/configs/ddl.xml create mode 100644 tests/integration/test_distributed_ddl_parallel/configs/dict.xml create mode 100644 tests/integration/test_distributed_ddl_parallel/configs/remote_servers.xml create mode 100644 tests/integration/test_distributed_ddl_parallel/test.py diff --git a/tests/integration/test_distributed_ddl_parallel/__init__.py b/tests/integration/test_distributed_ddl_parallel/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_distributed_ddl_parallel/configs/ddl.xml b/tests/integration/test_distributed_ddl_parallel/configs/ddl.xml new file mode 100644 index 00000000000..b926f99c687 --- /dev/null +++ b/tests/integration/test_distributed_ddl_parallel/configs/ddl.xml @@ -0,0 +1,5 @@ + + + 2 + + diff --git a/tests/integration/test_distributed_ddl_parallel/configs/dict.xml b/tests/integration/test_distributed_ddl_parallel/configs/dict.xml new file mode 100644 index 00000000000..610d55841a0 --- /dev/null +++ b/tests/integration/test_distributed_ddl_parallel/configs/dict.xml @@ -0,0 +1,26 @@ + + + + slow_dict + + + sleep 7 + TabSeparated + + + + + + + + id + + + value + String + + + + 0 + + diff --git a/tests/integration/test_distributed_ddl_parallel/configs/remote_servers.xml b/tests/integration/test_distributed_ddl_parallel/configs/remote_servers.xml new file mode 100644 index 00000000000..8ffa9f024d7 --- /dev/null +++ b/tests/integration/test_distributed_ddl_parallel/configs/remote_servers.xml @@ -0,0 +1,18 @@ + + + + + + n1 + 9000 + + + + + n2 + 9000 + + + + + diff --git a/tests/integration/test_distributed_ddl_parallel/test.py b/tests/integration/test_distributed_ddl_parallel/test.py new file mode 100644 index 00000000000..96530b111cb --- /dev/null +++ b/tests/integration/test_distributed_ddl_parallel/test.py @@ -0,0 +1,89 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name +# pylint: disable=line-too-long + +from functools import wraps +import threading +import time +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +def add_instance(name): + main_configs=[ + 'configs/ddl.xml', + 'configs/remote_servers.xml', + ] + dictionaries=[ + 'configs/dict.xml', + ] + return cluster.add_instance(name, + main_configs=main_configs, + dictionaries=dictionaries, + with_zookeeper=True) + +initiator = add_instance('initiator') +n1 = add_instance('n1') +n2 = add_instance('n2') + +@pytest.fixture(scope='module', autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +# verifies that functions executes longer then `sec` +def longer_then(sec): + def wrapper(func): + @wraps(func) + def inner(*args, **kwargs): + ts = time.time() + result = func(*args, **kwargs) + te = time.time() + took = te-ts + assert took >= sec + return result + return inner + return wrapper + +# It takes 7 seconds to load slow_dict. +def thread_reload_dictionary(): + initiator.query('SYSTEM RELOAD DICTIONARY ON CLUSTER cluster slow_dict') + +# NOTE: uses inner function to exclude slow start_cluster() from timeout. + +def test_dict_load(): + @pytest.mark.timeout(10) + @longer_then(7) + def inner_test(): + initiator.query('SYSTEM RELOAD DICTIONARY slow_dict') + inner_test() + +def test_all_in_parallel(): + @pytest.mark.timeout(10) + @longer_then(7) + def inner_test(): + threads = [] + for _ in range(2): + threads.append(threading.Thread(target=thread_reload_dictionary)) + for thread in threads: + thread.start() + for thread in threads: + thread.join() + inner_test() + +def test_two_in_parallel_two_queued(): + @pytest.mark.timeout(19) + @longer_then(14) + def inner_test(): + threads = [] + for _ in range(4): + threads.append(threading.Thread(target=thread_reload_dictionary)) + for thread in threads: + thread.start() + for thread in threads: + thread.join() + inner_test() From 7185b9a9452c958eca79e58f77873717b225343d Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Sat, 12 Sep 2020 19:59:03 +0300 Subject: [PATCH 074/273] add changes in RU files --- docs/en/operations/settings/settings.md | 2 +- docs/ru/operations/settings/settings.md | 57 ++++++++++++++++++ .../data-types/aggregatefunction.md | 5 ++ docs/ru/sql-reference/data-types/array.md | 5 ++ .../data-types/lowcardinality.md | 59 +++++++++++++++++++ docs/ru/sql-reference/data-types/nullable.md | 5 ++ docs/ru/sql-reference/data-types/tuple.md | 5 ++ .../functions/type-conversion-functions.md | 41 ++++++++++++- 8 files changed, 177 insertions(+), 2 deletions(-) create mode 100644 docs/ru/sql-reference/data-types/lowcardinality.md diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 76fcfa2a616..b1aad4d8e6a 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1815,7 +1815,7 @@ Default value: 8192. Turns on or turns off using of single dictionary for the data part. -By default, ClickHouse server monitors the size of dictionaries and if a dictionary overflows then the server starts to write the next one. To prohibit creating several dictionaries set `low_cardinality_use_single_dictionary_for_part = 1`. +By default, the ClickHouse server monitors the size of dictionaries and if a dictionary overflows then the server starts to write the next one. To prohibit creating several dictionaries set `low_cardinality_use_single_dictionary_for_part = 1`. Possible values: diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 2c6e0f05fb5..da1c56e3daf 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1616,6 +1616,63 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; - [Обработка значения NULL в операторе IN](../../sql-reference/operators/in.md#in-null-processing) +## low\_cardinality\_max\_dictionary\_size {#low_cardinality_max_dictionary_size} + +Задает максимальную длину строк в общем глобальном словаре для типа данных `LowCardinality`, который может быть записан в файловую систему хранилища. Настройка предоствращает проблемы с оперативной памятью в случае неограниченного увеличения словаря. Все данные, которые не могут быть закодированы из-за ограничения максимального размера словаря, ClickHouse записывает обычным способом. + +Допустимые значения: + +- Положительное целое число. + +Значение по умолчанию: 8192. + +## low\_cardinality\_use\_single\_dictionary\_for\_part {#low_cardinality_use_single_dictionary_for_part} + +Включает или выключает использование единого словаря для частей данных. + +По умолчанию сервер ClickHouse следит за размером словарей, и если словарь переполняется, сервер создает следующий. Чтобы запретить создание нескольких словарей, задайте настройку `low_cardinality_use_single_dictionary_for_part = 1`. + +Допустимые значения: + +- 1 — Создание нескольких словарей для частей данных запрещено. +- 0 — Создание нескольких словарей для частей данных не запрещено. + +Значение по умолчанию: 0. + +## low\_cardinality\_allow\_in\_native\_format {#low_cardinality_allow_in_native_format} + +Разрешает или запрещает использование типа данных `LowCardinality` с форматом данных [Native](../../interfaces/formats.md#native). + +Если использование типа `LowCardinality` ограничено, сервер CLickHouse преобразует столбцы `LowCardinality` в обычные столбцы для запросов `SELECT`, а обычные столбцы - в столбцы `LowCardinality` для запросов `INSERT`. + +В основном настройка используется для сторонних клиентов, не поддерживающих тип данных `LowCardinality`. + +Допустимые значения: + +- 1 — Использование `LowCardinality` не ограничено. +- 0 — Использование `LowCardinality` ограничено. + +Значение по умолчанию: 1. + +## allow\_suspicious\_low\_cardinality\_types {#allow_suspicious_low_cardinality_types} + +Разрешает или запрещает использование типа данных `LowCardinality` с типами данных с фиксированным размером 8 байт или меньше: числовые типы данных и `FixedString (8_bytes_or_less)`. + +Для небольших фиксированных значений использование `LowCardinality` обычно неэффективно, поскольку ClickHouse хранит числовой индекс для каждой строки. В результате: + +- Используется больше дискового пространства. +- Потребление ОЗУ увеличивается, в зависимости от размера словаря. +- Некоторые функции работают медленнее из-за дополнительных операций кодирования. + +Время слияния в таблицах на движке [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) также может увеличиться по описанным выше причинам. + +Допустимые значения: + +- 1 — Использование `LowCardinality` не ограничено. +- 0 — Использование `LowCardinality` ограничено. + +Значение по умолчанию: 0. + ## background_buffer_flush_schedule_pool_size {#background_buffer_flush_schedule_pool_size} Задает количество потоков для выполнения фонового сброса данных в таблицах с движком [Buffer](../../engines/table-engines/special/buffer.md). Настройка применяется при запуске сервера ClickHouse и не может быть изменена в пользовательском сеансе. diff --git a/docs/ru/sql-reference/data-types/aggregatefunction.md b/docs/ru/sql-reference/data-types/aggregatefunction.md index 07983885bde..22825deb3eb 100644 --- a/docs/ru/sql-reference/data-types/aggregatefunction.md +++ b/docs/ru/sql-reference/data-types/aggregatefunction.md @@ -1,3 +1,8 @@ +--- +toc_priority: 53 +toc_title: AggregateFunction +--- + # AggregateFunction {#data-type-aggregatefunction} Агрегатные функции могут обладать определяемым реализацией промежуточным состоянием, которое может быть сериализовано в тип данных, соответствующий AggregateFunction(…), и быть записано в таблицу обычно посредством [материализованного представления] (../../sql-reference/statements/create.md#create-view). Чтобы получить промежуточное состояние, обычно используются агрегатные функции с суффиксом `-State`. Чтобы в дальнейшем получить агрегированные данные необходимо использовать те же агрегатные функции с суффиксом `-Merge`. diff --git a/docs/ru/sql-reference/data-types/array.md b/docs/ru/sql-reference/data-types/array.md index 09973d8162c..906246b66ee 100644 --- a/docs/ru/sql-reference/data-types/array.md +++ b/docs/ru/sql-reference/data-types/array.md @@ -1,3 +1,8 @@ +--- +toc_priority: 52 +toc_title: Array(T) +--- + # Array(T) {#data-type-array} Массив из элементов типа `T`. diff --git a/docs/ru/sql-reference/data-types/lowcardinality.md b/docs/ru/sql-reference/data-types/lowcardinality.md new file mode 100644 index 00000000000..fc10624600a --- /dev/null +++ b/docs/ru/sql-reference/data-types/lowcardinality.md @@ -0,0 +1,59 @@ +--- +toc_priority: 51 +toc_title: LowCardinality +--- + +# LowCardinality {#lowcardinality-data-type} + +Изменяет внутреннее представление других типов данных, превращая их в тип со словарным кодированием. + +## Синтаксис {#lowcardinality-syntax} + +```sql +LowCardinality(data_type) +``` + +**Параметры** + +- `data_type` — [String](string.md), [FixedString](fixedstring.md), [Date](date.md), [DateTime](datetime.md) и числа за исключением типа [Decimal](decimal.md). `LowCardinality` неэффективен для некоторых типов данных, см. описание настройки [allow_suspicious_low_cardinality_types](../../operations/settings/settings.md#allow_suspicious_low_cardinality_types). + +## Описание {#lowcardinality-dscr} + +`LowCardinality` — это надстройка, изменяющая способ хранения и правила обработки данных. ClickHouse применяет [словарное кодирование](https://en.wikipedia.org/wiki/Dictionary_coder) в столбцы типа `LowCardinality`. Работа с данными, представленными в словарном виде, значительно увеличивает производительность запросов [SELECT](../statements/select/index.md) для многих приложений. + +Эффективность использования типа данных `LowCarditality` зависит от разнообразия данных. Если словарь содержит менее 10 000 различных значений, ClickHouse в основном показывает более высокую эффективность чтения и хранения данных. Если же словарь содержит более 100 000 различных значений, ClickHouse может работать хуже, чем при использовании обычных типов данных. + +При работе со строками используйте `LowCardinality` вместо [Enum](enum.md). `LowCardinality` обеспечивает большую гибкость в использовании и часто показывает такую же или более высокую эффективность. + +## Пример + +Создать таблицу со столбцами типа `LowCardinality`: + +```sql +CREATE TABLE lc_t +( + `id` UInt16, + `strings` LowCardinality(String) +) +ENGINE = MergeTree() +ORDER BY id +``` + +## Связанные настройки и функции + +Настройки: + +- [low_cardinality_max_dictionary_size](../../operations/settings/settings.md#low_cardinality_max_dictionary_size) +- [low_cardinality_use_single_dictionary_for_part](../../operations/settings/settings.md#low_cardinality_use_single_dictionary_for_part) +- [low_cardinality_allow_in_native_format](../../operations/settings/settings.md#low_cardinality_allow_in_native_format) +- [allow_suspicious_low_cardinality_types](../../operations/settings/settings.md#allow_suspicious_low_cardinality_types) + +Функции: + +- [toLowCardinality](../functions/type-conversion-functions.md#tolowcardinality) + +## Смотрите также + +- [A Magical Mystery Tour of the LowCardinality Data Type](https://www.altinity.com/blog/2019/3/27/low-cardinality). +- [Reducing Clickhouse Storage Cost with the Low Cardinality Type – Lessons from an Instana Engineer](https://www.instana.com/blog/reducing-clickhouse-storage-cost-with-the-low-cardinality-type-lessons-from-an-instana-engineer/). +- [String Optimization (video presentation in Russian)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [Slides in English](https://github.com/yandex/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf). \ No newline at end of file diff --git a/docs/ru/sql-reference/data-types/nullable.md b/docs/ru/sql-reference/data-types/nullable.md index 5ed99469750..71e1f7a37a0 100644 --- a/docs/ru/sql-reference/data-types/nullable.md +++ b/docs/ru/sql-reference/data-types/nullable.md @@ -1,3 +1,8 @@ +--- +toc_priority: 55 +toc_title: Nullable +--- + # Nullable(TypeName) {#data_type-nullable} Позволяет работать как со значением типа `TypeName` так и с отсутствием этого значения ([NULL](../../sql-reference/data-types/nullable.md)) в одной и той же переменной, в том числе хранить `NULL` в таблицах вместе со значения типа `TypeName`. Например, в столбце типа `Nullable(Int8)` можно хранить значения типа `Int8`, а в тех строках, где значения нет, будет храниться `NULL`. diff --git a/docs/ru/sql-reference/data-types/tuple.md b/docs/ru/sql-reference/data-types/tuple.md index 566a582eb95..cb8130f28a3 100644 --- a/docs/ru/sql-reference/data-types/tuple.md +++ b/docs/ru/sql-reference/data-types/tuple.md @@ -1,3 +1,8 @@ +--- +toc_priority: 54 +toc_title: Tuple(T1, T2, ...) +--- + # Tuple(T1, T2, …) {#tuplet1-t2} Кортеж из элементов любого [типа](index.md#data_types). Элементы кортежа могут быть одного или разных типов. diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 41ded78055c..3b70f0d6577 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -508,9 +508,48 @@ SELECT parseDateTimeBestEffort('10 20:19') **См. также** -- \[Информация о формате ISO 8601 от @xkcd\](https://xkcd.com/1179/) +- [Информация о формате ISO 8601 от @xkcd](https://xkcd.com/1179/) - [RFC 1123](https://tools.ietf.org/html/rfc1123) - [toDate](#todate) - [toDateTime](#todatetime) +## toLowCardinality {#tolowcardinality} + +Преобразует входные данные в версию [LowCardianlity](../data-types/lowcardinality.md) того же типа данных. + +Чтобы преобразовать данные из типа `LowCardinality`, используйте функцию [CAST](#type_conversion_function-cast). Например, `CAST(x as String)`. + +**Синтаксис** + +```sql +toLowCardinality(expr) +``` + +**Параметры** + +- `expr` — [Выражение](../syntax.md#syntax-expressions), которое в результате преобразуется в один из [поддерживаемых типов данных](../data-types/index.md#data_types). + + +**Возвращаемое значение** + +- Результат преобразования `expr`. + +Тип: `LowCardinality(expr_result_type)` + +**Example** + +Запрос: + +```sql +SELECT toLowCardinality('1') +``` + +Результат: + +```text +┌─toLowCardinality('1')─┐ +│ 1 │ +└───────────────────────┘ +``` + [Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/type_conversion_functions/) From 6ba9c107df75d21c9a4642c67b8552c9bcb7bcf2 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Sat, 12 Sep 2020 20:34:08 +0300 Subject: [PATCH 075/273] add EN changes --- docs/en/sql-reference/data-types/lowcardinality.md | 2 +- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/lowcardinality.md b/docs/en/sql-reference/data-types/lowcardinality.md index 7ccac61e4d7..1a0cedb99c7 100644 --- a/docs/en/sql-reference/data-types/lowcardinality.md +++ b/docs/en/sql-reference/data-types/lowcardinality.md @@ -21,7 +21,7 @@ LowCardinality(data_type) `LowCardinality` is a superstructure that changes a data storage method and rules of data processing. ClickHouse applies [dictionary coding](https://en.wikipedia.org/wiki/Dictionary_coder) to `LowCardinality`-columns. Operating with dictionary encoded data significantly increases performance of [SELECT](../../sql-reference/statements/select/index.md) queries for many applications. -The efficiency of using `LowCarditality` data type depends on data diversity. If a dictionary contains less than 10,000 distinct values, then ClickHouse mostly shows higher efficiency of data reading and storing. If a dictionary contains more than 100,000 distinct values, then ClickHouse can perform worse in comparison with using ordinary data types. +The efficiency of using `LowCardinality` data type depends on data diversity. If a dictionary contains less than 10,000 distinct values, then ClickHouse mostly shows higher efficiency of data reading and storing. If a dictionary contains more than 100,000 distinct values, then ClickHouse can perform worse in comparison with using ordinary data types. Consider using `LowCardinality` instead of [Enum](../../sql-reference/data-types/enum.md) when working with strings. `LowCardinality` provides more flexibility in use and often reveals the same or higher efficiency. diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 67361c350c7..e466c025d80 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -516,7 +516,7 @@ Result: **See Also** -- \[ISO 8601 announcement by @xkcd\](https://xkcd.com/1179/) +- [ISO 8601 announcement by @xkcd](https://xkcd.com/1179/) - [RFC 1123](https://tools.ietf.org/html/rfc1123) - [toDate](#todate) - [toDateTime](#todatetime) From 882b2a33488c4dbabbe96fff40c01a065fe0a860 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 13 Sep 2020 01:00:04 +0800 Subject: [PATCH 076/273] CTE --- src/Interpreters/ApplyWithSubqueryVisitor.cpp | 90 +++++++++++++++++++ src/Interpreters/ApplyWithSubqueryVisitor.h | 30 +++++++ .../ExecuteScalarSubqueriesVisitor.cpp | 5 ++ src/Interpreters/InterpreterSelectQuery.cpp | 3 + src/Interpreters/ya.make | 1 + src/Parsers/ASTWithElement.cpp | 21 +++++ src/Parsers/ASTWithElement.h | 25 ++++++ src/Parsers/ParserSelectQuery.cpp | 6 +- src/Parsers/ParserWithElement.cpp | 39 ++++++++ src/Parsers/ParserWithElement.h | 18 ++++ src/Parsers/ya.make | 2 + ...495_subqueries_in_with_statement.reference | 14 +++ .../01495_subqueries_in_with_statement.sql | 13 +++ 13 files changed, 266 insertions(+), 1 deletion(-) create mode 100644 src/Interpreters/ApplyWithSubqueryVisitor.cpp create mode 100644 src/Interpreters/ApplyWithSubqueryVisitor.h create mode 100644 src/Parsers/ASTWithElement.cpp create mode 100644 src/Parsers/ASTWithElement.h create mode 100644 src/Parsers/ParserWithElement.cpp create mode 100644 src/Parsers/ParserWithElement.h create mode 100644 tests/queries/0_stateless/01495_subqueries_in_with_statement.reference create mode 100644 tests/queries/0_stateless/01495_subqueries_in_with_statement.sql diff --git a/src/Interpreters/ApplyWithSubqueryVisitor.cpp b/src/Interpreters/ApplyWithSubqueryVisitor.cpp new file mode 100644 index 00000000000..e03682dafb3 --- /dev/null +++ b/src/Interpreters/ApplyWithSubqueryVisitor.cpp @@ -0,0 +1,90 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +void ApplyWithSubqueryVisitor::visit(ASTPtr & ast, const Data & data) +{ + if (auto * node_select = ast->as()) + { + auto with = node_select->with(); + std::optional new_data; + if (with) + { + for (auto & child : with->children) + visit(child, data); + for (auto & child : with->children) + { + if (auto * ast_with_elem = child->as()) + { + if (!new_data) + new_data = data; + new_data->subqueries[ast_with_elem->name] = ast_with_elem->subquery; + } + } + } + + for (auto & child : node_select->children) + { + if (child != with) + visit(child, new_data ? *new_data : data); + } + return; + } + + for (auto & child : ast->children) + visit(child, data); + if (auto * node_func = ast->as()) + visit(*node_func, data); + else if (auto * node_table = ast->as()) + visit(*node_table, data); +} + +void ApplyWithSubqueryVisitor::visit(ASTTableExpression & table, const Data & data) +{ + if (table.database_and_table_name) + { + auto table_id = IdentifierSemantic::extractDatabaseAndTable(table.database_and_table_name->as()); + if (table_id.database_name.empty()) + { + auto subquery_it = data.subqueries.find(table_id.table_name); + if (subquery_it != data.subqueries.end()) + { + table.children.clear(); + table.database_and_table_name.reset(); + table.subquery = subquery_it->second->clone(); + dynamic_cast(*table.subquery).alias = table_id.table_name; + table.children.emplace_back(table.subquery); + } + } + } +} + +void ApplyWithSubqueryVisitor::visit(ASTFunction & func, const Data & data) +{ + if (checkFunctionIsInOrGlobalInOperator(func)) + { + auto & ast = func.arguments->children.at(1); + if (const auto * ident = ast->as()) + { + auto table_id = IdentifierSemantic::extractDatabaseAndTable(*ident); + if (table_id.database_name.empty()) + { + auto subquery_it = data.subqueries.find(table_id.table_name); + if (subquery_it != data.subqueries.end()) + { + func.arguments->children[1] = subquery_it->second->clone(); + dynamic_cast(*func.arguments->children[1]).alias = table_id.table_name; + } + } + } + } +} + +} diff --git a/src/Interpreters/ApplyWithSubqueryVisitor.h b/src/Interpreters/ApplyWithSubqueryVisitor.h new file mode 100644 index 00000000000..2aecd6aee01 --- /dev/null +++ b/src/Interpreters/ApplyWithSubqueryVisitor.h @@ -0,0 +1,30 @@ +#pragma once + +#include + +#include + +namespace DB +{ +// TODO After we support `union_with_global`, this visitor should also be extended to match ASTSelectQueryWithUnion. +class ASTSelectQuery; +class ASTFunction; +struct ASTTableExpression; + +class ApplyWithSubqueryVisitor +{ +public: + struct Data + { + std::map subqueries; + }; + + static void visit(ASTPtr & ast) { visit(ast, {}); } + +private: + static void visit(ASTPtr & ast, const Data & data); + static void visit(ASTTableExpression & table, const Data & data); + static void visit(ASTFunction & func, const Data & data); +}; + +} diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index ee29d301c6b..f7a1fc83182 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -40,6 +41,10 @@ bool ExecuteScalarSubqueriesMatcher::needChildVisit(ASTPtr & node, const ASTPtr if (node->as()) return false; + /// Do not go to subqueries defined in with statement + if (node->as()) + return false; + if (node->as()) { /// Do not go to FROM, JOIN, UNION. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index dbf6d5ae8d3..603476ac1ba 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -14,6 +14,7 @@ #include +#include #include #include #include @@ -249,6 +250,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( source_header = input_pipe->getHeader(); } + ApplyWithSubqueryVisitor().visit(query_ptr); + JoinedTables joined_tables(getSubqueryContext(*context), getSelectQuery()); if (!has_input && !storage) diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index 4eacdab1dcf..5f520505a8a 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -23,6 +23,7 @@ SRCS( addTypeConversionToAST.cpp AggregateDescription.cpp Aggregator.cpp + ApplyWithSubqueryVisitor.cpp ArithmeticOperationsInAgrFuncOptimize.cpp ArrayJoinAction.cpp AsynchronousMetricLog.cpp diff --git a/src/Parsers/ASTWithElement.cpp b/src/Parsers/ASTWithElement.cpp new file mode 100644 index 00000000000..e8dd4ff0498 --- /dev/null +++ b/src/Parsers/ASTWithElement.cpp @@ -0,0 +1,21 @@ +#include + +namespace DB +{ + +ASTPtr ASTWithElement::clone() const +{ + const auto res = std::make_shared(*this); + res->name = name; + res->subquery = subquery->clone(); + res->children.emplace_back(res->subquery); + return res; +} + +void ASTWithElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + settings.writeIdentifier(name); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : ""); + subquery->formatImpl(settings, state, frame); +} +} diff --git a/src/Parsers/ASTWithElement.h b/src/Parsers/ASTWithElement.h new file mode 100644 index 00000000000..97c68579fa1 --- /dev/null +++ b/src/Parsers/ASTWithElement.h @@ -0,0 +1,25 @@ +#pragma once + +#include + + +namespace DB +{ +/** subquery in with statement + */ +class ASTWithElement : public IAST +{ +public: + String name; + ASTPtr subquery; + + /** Get the text that identifies this element. */ + String getID(char) const override { return "WithElement"; } + + ASTPtr clone() const override; + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +}; + +} diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index d2d7bbf9f21..9f2df82b4b4 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -74,7 +75,10 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (s_with.ignore(pos, expected)) { - if (!exp_list_for_with_clause.parse(pos, with_expression_list, expected)) + if (!ParserList(std::make_unique(), std::make_unique(TokenType::Comma)) + .parse(pos, with_expression_list, expected)) + return false; + if (with_expression_list->children.empty()) return false; } } diff --git a/src/Parsers/ParserWithElement.cpp b/src/Parsers/ParserWithElement.cpp new file mode 100644 index 00000000000..048e891f0df --- /dev/null +++ b/src/Parsers/ParserWithElement.cpp @@ -0,0 +1,39 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +bool ParserWithElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserIdentifier s_ident; + ParserKeyword s_as("AS"); + ParserSubquery s_subquery; + + auto old_pos = pos; + if (ASTPtr name, subquery; + s_ident.parse(pos, name, expected) && s_as.ignore(pos, expected) && s_subquery.parse(pos, subquery, expected)) + { + auto with_element = std::make_shared(); + tryGetIdentifierNameInto(name, with_element->name); + with_element->subquery = subquery; + node = with_element; + } + else + { + pos = old_pos; + ParserExpressionWithOptionalAlias s_expr(false); + if (!s_expr.parse(pos, node, expected)) + return false; + } + return true; +} + + +} diff --git a/src/Parsers/ParserWithElement.h b/src/Parsers/ParserWithElement.h new file mode 100644 index 00000000000..75ad11f5deb --- /dev/null +++ b/src/Parsers/ParserWithElement.h @@ -0,0 +1,18 @@ +#pragma once + +#include + + +namespace DB +{ +/** WITH (scalar query) AS identifier + * or WITH identifier AS (subquery) + */ +class ParserWithElement : public IParserBase +{ +protected: + const char * getName() const override { return "WITH element"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/ya.make b/src/Parsers/ya.make index fabf2bbb8fd..0a0c301b722 100644 --- a/src/Parsers/ya.make +++ b/src/Parsers/ya.make @@ -61,6 +61,7 @@ SRCS( ASTTTLElement.cpp ASTUserNameWithHost.cpp ASTWithAlias.cpp + ASTWithElement.cpp CommonParsers.cpp ExpressionElementParsers.cpp ExpressionListParsers.cpp @@ -133,6 +134,7 @@ SRCS( ParserUseQuery.cpp ParserUserNameWithHost.cpp ParserWatchQuery.cpp + ParserWithElement.cpp parseUserName.cpp queryToString.cpp QueryWithOutputSettingsPushDownVisitor.cpp diff --git a/tests/queries/0_stateless/01495_subqueries_in_with_statement.reference b/tests/queries/0_stateless/01495_subqueries_in_with_statement.reference new file mode 100644 index 00000000000..8e851cd3ba1 --- /dev/null +++ b/tests/queries/0_stateless/01495_subqueries_in_with_statement.reference @@ -0,0 +1,14 @@ +0 +1 +2 +3 +4 +2 3 +4 5 +2 3 +4 5 +1 1 2 +3 3 4 +4 5 +4 5 +4 5 diff --git a/tests/queries/0_stateless/01495_subqueries_in_with_statement.sql b/tests/queries/0_stateless/01495_subqueries_in_with_statement.sql new file mode 100644 index 00000000000..9ec921a9d4c --- /dev/null +++ b/tests/queries/0_stateless/01495_subqueries_in_with_statement.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS test1; + +CREATE TABLE test1(i int, j int) ENGINE Log; + +INSERT INTO test1 VALUES (1, 2), (3, 4); + +WITH test1 AS (SELECT * FROM numbers(5)) SELECT * FROM test1; +WITH test1 AS (SELECT i + 1, j + 1 FROM test1) SELECT * FROM test1; +WITH test1 AS (SELECT i + 1, j + 1 FROM test1) SELECT * FROM (SELECT * FROM test1); +SELECT * FROM (WITH test1 AS (SELECT toInt32(*) i FROM numbers(5)) SELECT * FROM test1) l ANY INNER JOIN test1 r on (l.i == r.i); +WITH test1 AS (SELECT i + 1, j + 1 FROM test1) SELECT toInt64(4) i, toInt64(5) j FROM numbers(3) WHERE (i, j) IN test1; + +DROP TABLE IF EXISTS test1; From a913be920117abad8e28039a69bacbe3d6307b6f Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 14 Sep 2020 19:36:14 +0800 Subject: [PATCH 077/273] 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 ac9ba23bdfa67bd0188ec00ccbff9816bc981bd5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 14 Sep 2020 15:49:04 +0300 Subject: [PATCH 078/273] fix more tests --- tests/integration/test_check_table/test.py | 3 ++- .../configs/wide_parts_only.xml | 6 ++++++ .../test_default_compression_codec/test.py | 6 +++--- .../configs/config.d/storage_conf.xml | 4 ++++ .../test_merge_tree_s3_with_cache/test.py | 21 ++++++++++--------- .../configs/wide_parts_only.xml | 6 ++++++ .../test_mutations_hardlinks/test.py | 2 +- 7 files changed, 33 insertions(+), 15 deletions(-) create mode 100644 tests/integration/test_default_compression_codec/configs/wide_parts_only.xml create mode 100644 tests/integration/test_mutations_hardlinks/configs/wide_parts_only.xml diff --git a/tests/integration/test_check_table/test.py b/tests/integration/test_check_table/test.py index 83df59b44a0..f972e7a92ba 100644 --- a/tests/integration/test_check_table/test.py +++ b/tests/integration/test_check_table/test.py @@ -24,7 +24,8 @@ def started_cluster(): node1.query(''' CREATE TABLE non_replicated_mt(date Date, id UInt32, value Int32) - ENGINE = MergeTree() PARTITION BY toYYYYMM(date) ORDER BY id; + ENGINE = MergeTree() PARTITION BY toYYYYMM(date) ORDER BY id + SETTINGS min_bytes_for_wide_part=0; ''') yield cluster diff --git a/tests/integration/test_default_compression_codec/configs/wide_parts_only.xml b/tests/integration/test_default_compression_codec/configs/wide_parts_only.xml new file mode 100644 index 00000000000..42e2173f718 --- /dev/null +++ b/tests/integration/test_default_compression_codec/configs/wide_parts_only.xml @@ -0,0 +1,6 @@ + + + 0 + 0 + + diff --git a/tests/integration/test_default_compression_codec/test.py b/tests/integration/test_default_compression_codec/test.py index d312a93ba01..0cfbb0b67cf 100644 --- a/tests/integration/test_default_compression_codec/test.py +++ b/tests/integration/test_default_compression_codec/test.py @@ -6,9 +6,9 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', main_configs=['configs/default_compression.xml'], with_zookeeper=True) -node2 = cluster.add_instance('node2', main_configs=['configs/default_compression.xml'], with_zookeeper=True) -node3 = cluster.add_instance('node3', main_configs=['configs/default_compression.xml'], image='yandex/clickhouse-server', tag='20.3.16', stay_alive=True, with_installed_binary=True) +node1 = cluster.add_instance('node1', main_configs=['configs/default_compression.xml', 'configs/wide_parts_only.xml'], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=['configs/default_compression.xml', 'configs/wide_parts_only.xml'], with_zookeeper=True) +node3 = cluster.add_instance('node3', main_configs=['configs/default_compression.xml', 'configs/wide_parts_only.xml'], image='yandex/clickhouse-server', tag='20.3.16', stay_alive=True, with_installed_binary=True) @pytest.fixture(scope="module") def start_cluster(): diff --git a/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/storage_conf.xml index b32770095fc..f3b7f959ce9 100644 --- a/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3_with_cache/configs/config.d/storage_conf.xml @@ -18,4 +18,8 @@ + + + 0 + diff --git a/tests/integration/test_merge_tree_s3_with_cache/test.py b/tests/integration/test_merge_tree_s3_with_cache/test.py index 25c08777ae5..d5d6db2fb77 100644 --- a/tests/integration/test_merge_tree_s3_with_cache/test.py +++ b/tests/integration/test_merge_tree_s3_with_cache/test.py @@ -40,7 +40,8 @@ def get_query_stat(instance, hint): return result -def test_write_is_cached(cluster): +@pytest.mark.parametrize("min_rows_for_wide_part,read_requests", [(0, 2), (8192, 1)]) +def test_write_is_cached(cluster, min_rows_for_wide_part, read_requests): node = cluster.instances["node"] node.query( @@ -50,8 +51,8 @@ def test_write_is_cached(cluster): data String ) ENGINE=MergeTree() ORDER BY id - SETTINGS storage_policy='s3' - """ + SETTINGS storage_policy='s3', min_rows_for_wide_part={} + """.format(min_rows_for_wide_part) ) node.query("SYSTEM FLUSH LOGS") @@ -63,12 +64,12 @@ def test_write_is_cached(cluster): assert node.query(select_query) == "(0,'data'),(1,'data')" stat = get_query_stat(node, select_query) - assert stat["S3ReadRequestsCount"] == 2 # Only .bin files should be accessed from S3. + assert stat["S3ReadRequestsCount"] == read_requests # Only .bin files should be accessed from S3. node.query("DROP TABLE IF EXISTS s3_test NO DELAY") - -def test_read_after_cache_is_wiped(cluster): +@pytest.mark.parametrize("min_rows_for_wide_part,all_files,bin_files", [(0, 4, 2), (8192, 2, 1)]) +def test_read_after_cache_is_wiped(cluster, min_rows_for_wide_part, all_files, bin_files): node = cluster.instances["node"] node.query( @@ -78,8 +79,8 @@ def test_read_after_cache_is_wiped(cluster): data String ) ENGINE=MergeTree() ORDER BY id - SETTINGS storage_policy='s3' - """ + SETTINGS storage_policy='s3', min_rows_for_wide_part={} + """.format(min_rows_for_wide_part) ) node.query("SYSTEM FLUSH LOGS") @@ -93,12 +94,12 @@ def test_read_after_cache_is_wiped(cluster): select_query = "SELECT * FROM s3_test" node.query(select_query) stat = get_query_stat(node, select_query) - assert stat["S3ReadRequestsCount"] == 4 # .mrk and .bin files should be accessed from S3. + assert stat["S3ReadRequestsCount"] == all_files # .mrk and .bin files should be accessed from S3. # After cache is populated again, only .bin files should be accessed from S3. select_query = "SELECT * FROM s3_test order by id FORMAT Values" assert node.query(select_query) == "(0,'data'),(1,'data')" stat = get_query_stat(node, select_query) - assert stat["S3ReadRequestsCount"] == 2 + assert stat["S3ReadRequestsCount"] == bin_files node.query("DROP TABLE IF EXISTS s3_test NO DELAY") diff --git a/tests/integration/test_mutations_hardlinks/configs/wide_parts_only.xml b/tests/integration/test_mutations_hardlinks/configs/wide_parts_only.xml new file mode 100644 index 00000000000..42e2173f718 --- /dev/null +++ b/tests/integration/test_mutations_hardlinks/configs/wide_parts_only.xml @@ -0,0 +1,6 @@ + + + 0 + 0 + + diff --git a/tests/integration/test_mutations_hardlinks/test.py b/tests/integration/test_mutations_hardlinks/test.py index 56852f572ff..4e70e76bc63 100644 --- a/tests/integration/test_mutations_hardlinks/test.py +++ b/tests/integration/test_mutations_hardlinks/test.py @@ -9,7 +9,7 @@ from multiprocessing.dummy import Pool cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1') +node1 = cluster.add_instance('node1', main_configs=['configs/wide_parts_only.xml']) @pytest.fixture(scope="module") def started_cluster(): From 5697f6d926c2dc04892aca3ef7b8297ef91d8da6 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Mon, 14 Sep 2020 16:14:18 +0300 Subject: [PATCH 079/273] style fix --- src/Columns/ColumnVector.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index c548ce3ca5c..4d7b7856363 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -19,7 +19,6 @@ #include - #if !defined(ARCADIA_BUILD) # include # if USE_OPENCL @@ -219,7 +218,7 @@ void ColumnVector::getPermutation(bool reverse, size_t limit, int nan_directi if (isNaN(data[res[reverse ? i : s - 1 - i]])) ++nans_to_move; else - break; + break;completeThread } if (nans_to_move) From 2bffefae1ac4ceb5fe48a5b445a6b1ac4dfe6ff7 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Mon, 14 Sep 2020 16:30:44 +0300 Subject: [PATCH 080/273] typo --- src/Columns/ColumnVector.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 4d7b7856363..a09f64ad580 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -218,7 +218,7 @@ void ColumnVector::getPermutation(bool reverse, size_t limit, int nan_directi if (isNaN(data[res[reverse ? i : s - 1 - i]])) ++nans_to_move; else - break;completeThread + break; } if (nans_to_move) From 3795dfed144c93e57486c7d0ab5d370e9e8cc82b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Sep 2020 16:33:36 +0300 Subject: [PATCH 081/273] 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 082/273] 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 083/273] 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 f725f8deee7fb8d695e1e3282a8b830a95ccf6ed Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 15 Sep 2020 02:14:14 +0300 Subject: [PATCH 084/273] fix more tests --- .../test_backward_compatibility/configs/wide_parts_only.xml | 5 +++++ tests/integration/test_backward_compatibility/test.py | 4 ++-- .../0_stateless/00804_test_alter_compression_codecs.sql | 2 +- .../0_stateless/00926_adaptive_index_granularity_pk.sql | 2 +- 4 files changed, 9 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_backward_compatibility/configs/wide_parts_only.xml diff --git a/tests/integration/test_backward_compatibility/configs/wide_parts_only.xml b/tests/integration/test_backward_compatibility/configs/wide_parts_only.xml new file mode 100644 index 00000000000..b240c0fcb2a --- /dev/null +++ b/tests/integration/test_backward_compatibility/configs/wide_parts_only.xml @@ -0,0 +1,5 @@ + + + 0 + + diff --git a/tests/integration/test_backward_compatibility/test.py b/tests/integration/test_backward_compatibility/test.py index 5b51823d361..cef70add3d0 100644 --- a/tests/integration/test_backward_compatibility/test.py +++ b/tests/integration/test_backward_compatibility/test.py @@ -5,7 +5,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', with_zookeeper=True, image='yandex/clickhouse-server', tag='19.17.8.54', stay_alive=True, with_installed_binary=True) -node2 = cluster.add_instance('node2', with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=['configs/wide_parts_only.xml'], with_zookeeper=True) @pytest.fixture(scope="module") def start_cluster(): @@ -24,7 +24,7 @@ def start_cluster(): cluster.shutdown() -def test_backward_compatability(start_cluster): +def test_backward_compatability1(start_cluster): node2.query("INSERT INTO t VALUES (today(), 1)") node1.query("SYSTEM SYNC REPLICA t", timeout=10) diff --git a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql index a9e6c12735c..4710694baf5 100644 --- a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql @@ -63,7 +63,7 @@ CREATE TABLE large_alter_table_00804 ( somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12)), id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC), data String CODEC(ZSTD(2), LZ4HC, NONE, LZ4, LZ4) -) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; +) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2 SETTINGS min_bytes_for_wide_part = 0; INSERT INTO large_alter_table_00804 SELECT toDate('2019-01-01'), number, toString(number + rand()) FROM system.numbers LIMIT 300000; diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_pk.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_pk.sql index b71c0640bd5..fe434845c29 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_pk.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_pk.sql @@ -62,7 +62,7 @@ CREATE TABLE large_alter_table_00926 ( somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12)), id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC), data String CODEC(ZSTD(2), LZ4HC, NONE, LZ4, LZ4) -) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity_bytes=40, min_index_granularity_bytes=30, write_final_mark = 0; +) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS min_index_granularity_bytes=30, write_final_mark = 0, min_bytes_for_wide_part = '10M'; INSERT INTO large_alter_table_00926 SELECT toDate('2019-01-01'), number, toString(number + rand()) FROM system.numbers LIMIT 300000; From 52f921a6f98772ba75d34df77195a840fec37758 Mon Sep 17 00:00:00 2001 From: Sergei Shtykov Date: Tue, 15 Sep 2020 12:24:03 +0300 Subject: [PATCH 085/273] CLICKHOUSEDOCS-758: Fixed links --- docs/ru/interfaces/formats.md | 4 ++-- docs/ru/operations/settings/settings.md | 2 +- .../aggregate-functions/reference/groupbitmap.md | 2 +- docs/ru/sql-reference/functions/bitmap-functions.md | 8 ++++---- docs/ru/sql-reference/functions/random-functions.md | 1 + docs/tools/test.py | 2 +- 6 files changed, 10 insertions(+), 9 deletions(-) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 04bca115974..dd68f7eb646 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -1050,13 +1050,13 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Parquet" > {some_ Для обмена данными с экосистемой Hadoop можно использовать движки таблиц [HDFS](../engines/table-engines/integrations/hdfs.md). -## Arrow {data-format-arrow} +## Arrow {#data-format-arrow} [Apache Arrow](https://arrow.apache.org/) поставляется с двумя встроенными поколоночнами форматами хранения. ClickHouse поддерживает операции чтения и записи для этих форматов. `Arrow` — это Apache Arrow's "file mode" формат. Он предназначен для произвольного доступа в памяти. -## ArrowStream {data-format-arrow-stream} +## ArrowStream {#data-format-arrow-stream} `ArrowStream` — это Apache Arrow's "stream mode" формат. Он предназначен для обработки потоков в памяти. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 2c6e0f05fb5..333c827fe97 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -484,7 +484,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( См. также: -- [JOIN strictness](../../sql-reference/statements/select/join.md#select-join-strictness) +- [JOIN strictness](../../sql-reference/statements/select/join.md#join-settings) ## max\_block\_size {#setting-max_block_size} diff --git a/docs/ru/sql-reference/aggregate-functions/reference/groupbitmap.md b/docs/ru/sql-reference/aggregate-functions/reference/groupbitmap.md index c01636e155d..a4be18b75ec 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/groupbitmap.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/groupbitmap.md @@ -4,7 +4,7 @@ toc_priority: 128 # groupBitmap {#groupbitmap} -Bitmap или агрегатные вычисления для столбца с типом данных `UInt*`, возвращают кардинальность в виде значения типа UInt64, если добавить суффикс -State, то возвращают [объект bitmap](../../../sql-reference/functions/bitmap-functions.md). +Bitmap или агрегатные вычисления для столбца с типом данных `UInt*`, возвращают кардинальность в виде значения типа UInt64, если добавить суффикс `-State`, то возвращают [объект bitmap](../../../sql-reference/functions/bitmap-functions.md#bitmap-functions). ``` sql groupBitmap(expr) diff --git a/docs/ru/sql-reference/functions/bitmap-functions.md b/docs/ru/sql-reference/functions/bitmap-functions.md index c91725c7a39..c5b0646aa79 100644 --- a/docs/ru/sql-reference/functions/bitmap-functions.md +++ b/docs/ru/sql-reference/functions/bitmap-functions.md @@ -1,4 +1,4 @@ -# Функции для битмапов {#funktsii-dlia-bitmapov} +# Функции для битмапов {#bitmap-functions} ## bitmapBuild {#bitmap_functions-bitmapbuild} @@ -61,8 +61,8 @@ bitmapSubsetLimit(bitmap, range_start, cardinality_limit) **Параметры** - `bitmap` – Битмап. [Bitmap object](#bitmap_functions-bitmapbuild). -- `range_start` – Начальная точка подмножества. [UInt32](../../sql-reference/functions/bitmap-functions.md). -- `cardinality_limit` – Верхний предел подмножества. [UInt32](../../sql-reference/functions/bitmap-functions.md). +- `range_start` – Начальная точка подмножества. [UInt32](../../sql-reference/functions/bitmap-functions.md#bitmap-functions). +- `cardinality_limit` – Верхний предел подмножества. [UInt32](../../sql-reference/functions/bitmap-functions.md#bitmap-functions). **Возвращаемое значение** @@ -97,7 +97,7 @@ bitmapContains(haystack, needle) **Параметры** - `haystack` – [объект Bitmap](#bitmap_functions-bitmapbuild), в котором функция ищет значение. -- `needle` – значение, которое функция ищет. Тип — [UInt32](../../sql-reference/functions/bitmap-functions.md). +- `needle` – значение, которое функция ищет. Тип — [UInt32](../../sql-reference/functions/bitmap-functions.md#bitmap-functions). **Возвращаемые значения** diff --git a/docs/ru/sql-reference/functions/random-functions.md b/docs/ru/sql-reference/functions/random-functions.md index 4aaaef5cb5d..21dcfeeb3c0 100644 --- a/docs/ru/sql-reference/functions/random-functions.md +++ b/docs/ru/sql-reference/functions/random-functions.md @@ -100,5 +100,6 @@ FROM numbers(3) │ a*cjab+ │ │ aeca2A │ └───────────────────────────────────────┘ +``` [Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/random_functions/) diff --git a/docs/tools/test.py b/docs/tools/test.py index 5c0cf4b799d..d963d34df08 100755 --- a/docs/tools/test.py +++ b/docs/tools/test.py @@ -92,7 +92,7 @@ def test_single_page(input_path, lang): logging.warning('Found %d duplicate anchor points' % duplicate_anchor_points) if links_to_nowhere: - if lang == 'en': # TODO: check all languages again + if lang == 'en' or lang == 'ru': # TODO: check all languages again logging.error(f'Found {links_to_nowhere} links to nowhere in {lang}') sys.exit(1) else: From 03346a0a3024288850cd24e91884d5e0cb5889fe Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 15 Sep 2020 17:35:38 +0800 Subject: [PATCH 086/273] 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 d943bac1a482276812ac03b6ba161dc7f4bab648 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 15 Sep 2020 13:29:47 +0300 Subject: [PATCH 087/273] Exception on double init of global thread pool --- src/Common/ThreadPool.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 49516d777fb..93aa6be8d9a 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -13,6 +13,7 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_SCHEDULE_TASK; + extern const int LOGICAL_ERROR; } } @@ -276,7 +277,11 @@ std::unique_ptr GlobalThreadPool::the_instance; void GlobalThreadPool::initialize(size_t max_threads) { - assert(!the_instance); + if (the_instance) + { + throw Exception(LOGICAL_ERROR, + "The global thread pool is initialized twice"); + } the_instance.reset(new GlobalThreadPool(max_threads, 1000 /*max_free_threads*/, 10000 /*max_queue_size*/, 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 088/273] 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 106e05ab2f36e7222be3ef5af8ae156502480dba Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 15 Sep 2020 20:28:42 +0800 Subject: [PATCH 089/273] Try fix mysql protocol parse failure --- src/Core/MySQL/MySQLReplication.cpp | 92 ++++++++++--------- src/Core/MySQL/MySQLReplication.h | 2 +- src/IO/MySQLBinlogEventReadBuffer.cpp | 61 ++++++++++++ src/IO/MySQLBinlogEventReadBuffer.h | 26 ++++++ .../gtest_mysql_binlog_event_read_buffer.cpp | 20 ++++ src/IO/ya.make | 1 + 6 files changed, 160 insertions(+), 42 deletions(-) create mode 100644 src/IO/MySQLBinlogEventReadBuffer.cpp create mode 100644 src/IO/MySQLBinlogEventReadBuffer.h create mode 100644 src/IO/tests/gtest_mysql_binlog_event_read_buffer.cpp diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index e7f113ba7af..81d46d10025 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -100,9 +101,7 @@ namespace MySQLReplication payload.readStrict(reinterpret_cast(schema.data()), schema_len); payload.ignore(1); - size_t len = payload.available() - CHECKSUM_CRC32_SIGNATURE_LENGTH; - query.resize(len); - payload.readStrict(reinterpret_cast(query.data()), len); + readStringUntilEOF(query, payload); if (query.starts_with("BEGIN") || query.starts_with("COMMIT")) { typ = QUERY_EVENT_MULTI_TXN_FLAG; @@ -285,7 +284,7 @@ namespace MySQLReplication break; } - while (payload.available() > CHECKSUM_CRC32_SIGNATURE_LENGTH) + while (!payload.eof()) { parseRow(payload, columns_present_bitmap1); if (header.type == UPDATE_ROWS_EVENT_V1 || header.type == UPDATE_ROWS_EVENT_V2) @@ -738,7 +737,7 @@ namespace MySQLReplication payload.readStrict(reinterpret_cast(>id.seq_no), 8); /// Skip others. - payload.ignore(payload.available() - CHECKSUM_CRC32_SIGNATURE_LENGTH); + payload.ignoreAll(); } void GTIDEvent::dump(std::ostream & out) const @@ -804,46 +803,51 @@ namespace MySQLReplication void MySQLFlavor::readPayloadImpl(ReadBuffer & payload) { - UInt16 header = static_cast(*payload.position()); + MySQLBinlogEventReadBuffer event_payload(payload); + UInt16 header = static_cast(*event_payload.position()); switch (header) { case PACKET_EOF: throw ReplicationError("Master maybe lost", ErrorCodes::UNKNOWN_EXCEPTION); case PACKET_ERR: ERRPacket err; - err.readPayloadWithUnpacked(payload); + err.readPayloadWithUnpacked(event_payload); throw ReplicationError(err.error_message, ErrorCodes::UNKNOWN_EXCEPTION); } // skip the header flag. - payload.ignore(1); + event_payload.ignore(1); - EventType event_type = static_cast(*(payload.position() + 4)); + EventType event_type = static_cast(*(event_payload.position() + 4)); switch (event_type) { - case FORMAT_DESCRIPTION_EVENT: { + case FORMAT_DESCRIPTION_EVENT: + { event = std::make_shared(); - event->parseHeader(payload); - event->parseEvent(payload); + event->parseHeader(event_payload); + event->parseEvent(event_payload); position.update(event); break; } - case ROTATE_EVENT: { + case ROTATE_EVENT: + { event = std::make_shared(); - event->parseHeader(payload); - event->parseEvent(payload); + event->parseHeader(event_payload); + event->parseEvent(event_payload); position.update(event); break; } - case QUERY_EVENT: { + case QUERY_EVENT: + { event = std::make_shared(); - event->parseHeader(payload); - event->parseEvent(payload); + event->parseHeader(event_payload); + event->parseEvent(event_payload); auto query = std::static_pointer_cast(event); switch (query->typ) { case QUERY_EVENT_MULTI_TXN_FLAG: - case QUERY_EVENT_XA: { + case QUERY_EVENT_XA: + { event = std::make_shared(); break; } @@ -852,68 +856,74 @@ namespace MySQLReplication } break; } - case XID_EVENT: { + case XID_EVENT: + { event = std::make_shared(); - event->parseHeader(payload); - event->parseEvent(payload); + event->parseHeader(event_payload); + event->parseEvent(event_payload); position.update(event); break; } - case TABLE_MAP_EVENT: { + case TABLE_MAP_EVENT: + { event = std::make_shared(); - event->parseHeader(payload); - event->parseEvent(payload); + event->parseHeader(event_payload); + event->parseEvent(event_payload); table_map = std::static_pointer_cast(event); break; } case WRITE_ROWS_EVENT_V1: - case WRITE_ROWS_EVENT_V2: { + case WRITE_ROWS_EVENT_V2: + { if (do_replicate()) event = std::make_shared(table_map); else event = std::make_shared(); - event->parseHeader(payload); - event->parseEvent(payload); + event->parseHeader(event_payload); + event->parseEvent(event_payload); break; } case DELETE_ROWS_EVENT_V1: - case DELETE_ROWS_EVENT_V2: { + case DELETE_ROWS_EVENT_V2: + { if (do_replicate()) event = std::make_shared(table_map); else event = std::make_shared(); - event->parseHeader(payload); - event->parseEvent(payload); + event->parseHeader(event_payload); + event->parseEvent(event_payload); break; } case UPDATE_ROWS_EVENT_V1: - case UPDATE_ROWS_EVENT_V2: { + case UPDATE_ROWS_EVENT_V2: + { if (do_replicate()) event = std::make_shared(table_map); else event = std::make_shared(); - event->parseHeader(payload); - event->parseEvent(payload); + event->parseHeader(event_payload); + event->parseEvent(event_payload); break; } - case GTID_EVENT: { + case GTID_EVENT: + { event = std::make_shared(); - event->parseHeader(payload); - event->parseEvent(payload); + event->parseHeader(event_payload); + event->parseEvent(event_payload); position.update(event); break; } - default: { + default: + { event = std::make_shared(); - event->parseHeader(payload); - event->parseEvent(payload); + event->parseHeader(event_payload); + event->parseEvent(event_payload); break; } } - payload.ignoreAll(); } } diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index b63b103e87a..230055902e5 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -505,7 +505,7 @@ namespace MySQLReplication class MySQLFlavor : public IFlavor { public: - void readPayloadImpl(ReadBuffer & payload) override; + void readPayloadImpl(ReadBuffer & event_payload) override; String getName() const override { return "MySQL"; } Position getPosition() const override { return position; } BinlogEventPtr readOneEvent() override { return event; } diff --git a/src/IO/MySQLBinlogEventReadBuffer.cpp b/src/IO/MySQLBinlogEventReadBuffer.cpp new file mode 100644 index 00000000000..3b76efa64ac --- /dev/null +++ b/src/IO/MySQLBinlogEventReadBuffer.cpp @@ -0,0 +1,61 @@ +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +MySQLBinlogEventReadBuffer::MySQLBinlogEventReadBuffer(ReadBuffer & in_) + : ReadBuffer(nullptr, 0, 0), in(in_) +{ +} + +bool MySQLBinlogEventReadBuffer::nextImpl() +{ + if (hasPendingData()) + return true; + + if (in.eof()) + return false; + + if (likely(in.available() > CHECKSUM_CRC32_SIGNATURE_LENGTH)) + { + working_buffer = ReadBuffer::Buffer(in.position(), in.buffer().end() - CHECKSUM_CRC32_SIGNATURE_LENGTH); + in.ignore(working_buffer.size()); + return true; + } + + if (checksum_buff_size == checksum_buff_limit) + { + in.readStrict(checksum_buf, CHECKSUM_CRC32_SIGNATURE_LENGTH); + checksum_buff_size = checksum_buff_limit = CHECKSUM_CRC32_SIGNATURE_LENGTH; + } + else + { + for (size_t index = 0; index < checksum_buff_size - checksum_buff_limit; ++index) + checksum_buf[index] = checksum_buf[checksum_buff_limit + index]; + + checksum_buff_size -= checksum_buff_limit; + size_t read_bytes = CHECKSUM_CRC32_SIGNATURE_LENGTH - checksum_buff_size; + in.readStrict(checksum_buf + checksum_buff_size, read_bytes); /// Minimum CHECKSUM_CRC32_SIGNATURE_LENGTH bytes + checksum_buff_size = checksum_buff_limit = CHECKSUM_CRC32_SIGNATURE_LENGTH; + } + + if (in.eof()) + return false; + + if (in.available() < CHECKSUM_CRC32_SIGNATURE_LENGTH) + { + size_t left_move_size = CHECKSUM_CRC32_SIGNATURE_LENGTH - in.available(); + checksum_buff_limit = checksum_buff_size - left_move_size; + } + + working_buffer = ReadBuffer::Buffer(checksum_buf, checksum_buf + checksum_buff_limit); + return true; +} + +} diff --git a/src/IO/MySQLBinlogEventReadBuffer.h b/src/IO/MySQLBinlogEventReadBuffer.h new file mode 100644 index 00000000000..7a19461e57e --- /dev/null +++ b/src/IO/MySQLBinlogEventReadBuffer.h @@ -0,0 +1,26 @@ +#pragma once + +#include + +namespace DB +{ + +class MySQLBinlogEventReadBuffer : public ReadBuffer +{ +protected: + static const size_t CHECKSUM_CRC32_SIGNATURE_LENGTH = 4; + ReadBuffer & in; + + size_t checksum_buff_size = 0; + size_t checksum_buff_limit = 0; + char checksum_buf[CHECKSUM_CRC32_SIGNATURE_LENGTH]; + + bool nextImpl() override; + +public: + MySQLBinlogEventReadBuffer(ReadBuffer & in_); + +}; + + +} diff --git a/src/IO/tests/gtest_mysql_binlog_event_read_buffer.cpp b/src/IO/tests/gtest_mysql_binlog_event_read_buffer.cpp new file mode 100644 index 00000000000..183da5182af --- /dev/null +++ b/src/IO/tests/gtest_mysql_binlog_event_read_buffer.cpp @@ -0,0 +1,20 @@ +#include +#include +#include +#include + +using namespace DB; + +TEST(MySQLBinlogEventReadBuffer, CheckBoundary) +{ + for (size_t index = 1; index < 4; ++index) + { + std::vector memory_data(index, 0x01); + ReadBufferFromMemory nested_in(memory_data.data(), index); + + MySQLBinlogEventReadBuffer binlog_in(nested_in); + EXPECT_THROW(binlog_in.ignore(), Exception); + } +} + + diff --git a/src/IO/ya.make b/src/IO/ya.make index 0c939588a9b..28099818b46 100644 --- a/src/IO/ya.make +++ b/src/IO/ya.make @@ -28,6 +28,7 @@ SRCS( MemoryReadWriteBuffer.cpp MMapReadBufferFromFile.cpp MMapReadBufferFromFileDescriptor.cpp + MySQLBinlogEventReadBuffer.cpp MySQLPacketPayloadReadBuffer.cpp MySQLPacketPayloadWriteBuffer.cpp NullWriteBuffer.cpp From 63db2ca68d9797d5c6f5242350d55fdff6c88f05 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 15 Sep 2020 15:30:07 +0300 Subject: [PATCH 090/273] fix test --- .../queries/0_stateless/00804_test_alter_compression_codecs.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql index 4710694baf5..2a1b9e55b9a 100644 --- a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql @@ -63,7 +63,7 @@ CREATE TABLE large_alter_table_00804 ( somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12)), id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC), data String CODEC(ZSTD(2), LZ4HC, NONE, LZ4, LZ4) -) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2 SETTINGS min_bytes_for_wide_part = 0; +) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2, min_bytes_for_wide_part = 0; INSERT INTO large_alter_table_00804 SELECT toDate('2019-01-01'), number, toString(number + rand()) FROM system.numbers LIMIT 300000; From 4c783f19ee388fd7e5622e1585f91adb004cff29 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 15 Sep 2020 16:25:14 +0300 Subject: [PATCH 091/273] Use QueryPlan for SubqueryForSet. --- src/Interpreters/ActionsVisitor.cpp | 5 +- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 3 +- src/Interpreters/InterpreterSelectQuery.cpp | 38 +++++++-- src/Interpreters/SubqueryForSet.cpp | 5 +- src/Interpreters/SubqueryForSet.h | 4 +- src/Processors/QueryPipeline.cpp | 16 +++- src/Processors/QueryPipeline.h | 2 + src/Processors/QueryPlan/CreatingSetsStep.cpp | 84 +++++++++++++++---- src/Processors/QueryPlan/CreatingSetsStep.h | 26 +++++- 10 files changed, 150 insertions(+), 35 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 9d6d5f783ff..be040ff2c34 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -900,10 +900,11 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su * in the subquery_for_set object, this subquery is set as source and the temporary table _data1 as the table. * - this function shows the expression IN_data1. */ - if (subquery_for_set.source.empty() && data.no_storage_or_local) + if (!subquery_for_set.source && data.no_storage_or_local) { auto interpreter = interpretSubquery(right_in_operand, data.context, data.subquery_depth, {}); - subquery_for_set.source = QueryPipeline::getPipe(interpreter->execute().pipeline); + subquery_for_set.source = std::make_unique(); + interpreter->buildQueryPlan(*subquery_for_set.source); } subquery_for_set.set = set; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index d9fc44d9b8f..14a50c2cfc6 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -582,7 +582,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin(const ASTTablesInSelectQuer ExpressionActionsPtr joined_block_actions = createJoinedBlockActions(context, analyzedJoin()); Names original_right_columns; - if (subquery_for_join.source.empty()) + if (!subquery_for_join.source) { NamesWithAliases required_columns_with_aliases = analyzedJoin().getRequiredColumns( joined_block_actions->getSampleBlock(), joined_block_actions->getRequiredColumns()); diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index e155a132241..719794f0607 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -135,7 +135,8 @@ public: ast = database_and_table_name; external_tables[external_table_name] = external_storage_holder; - subqueries_for_sets[external_table_name].source = QueryPipeline::getPipe(interpreter->execute().pipeline); + subqueries_for_sets[external_table_name].source = std::make_unique(); + interpreter->buildQueryPlan(*subqueries_for_sets[external_table_name].source); subqueries_for_sets[external_table_name].table = external_storage; /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 079fc792447..6f1be43b3a1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1862,14 +1862,38 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_p const Settings & settings = context->getSettingsRef(); - auto creating_sets = std::make_unique( - query_plan.getCurrentDataStream(), - std::move(subqueries_for_sets), - SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), - *context); + if (subqueries_for_sets.empty()) + return; - creating_sets->setStepDescription("Create sets for subqueries and joins"); - query_plan.addStep(std::move(creating_sets)); + SizeLimits limits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode); + + std::vector plans; + DataStreams input_streams; + input_streams.emplace_back(query_plan.getCurrentDataStream()); + + for (auto & [description, set] : subqueries_for_sets) + { + auto plan = std::move(set.source); + std::string type = (set.join != nullptr) ? "JOIN" + : "subquery"; + + auto creating_set = std::make_unique( + plan->getCurrentDataStream(), + query_plan.getCurrentDataStream().header, + std::move(description), + std::move(set), + limits, + *context); + creating_set->setStepDescription("Create set for " + type); + plan->addStep(std::move(creating_set)); + + input_streams.emplace_back(plan->getCurrentDataStream()); + plans.emplace_back(std::move(*plan)); + } + + auto creating_sets = std::make_unique(std::move(input_streams)); + creating_sets->setStepDescription("Create sets before main query execution"); + query_plan.unitePlans(std::move(creating_sets), std::move(plans)); } diff --git a/src/Interpreters/SubqueryForSet.cpp b/src/Interpreters/SubqueryForSet.cpp index 038ecbbb0b6..e944b76e71c 100644 --- a/src/Interpreters/SubqueryForSet.cpp +++ b/src/Interpreters/SubqueryForSet.cpp @@ -12,9 +12,10 @@ void SubqueryForSet::makeSource(std::shared_ptr NamesWithAliases && joined_block_aliases_) { joined_block_aliases = std::move(joined_block_aliases_); - source = QueryPipeline::getPipe(interpreter->execute().pipeline); + source = std::make_unique(); + interpreter->buildQueryPlan(*source); - sample_block = source.getHeader(); + sample_block = interpreter->getSampleBlock(); renameColumns(sample_block); } diff --git a/src/Interpreters/SubqueryForSet.h b/src/Interpreters/SubqueryForSet.h index d268758c3e8..b44f0f6cf8b 100644 --- a/src/Interpreters/SubqueryForSet.h +++ b/src/Interpreters/SubqueryForSet.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB @@ -14,12 +13,13 @@ namespace DB class InterpreterSelectWithUnionQuery; class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; +class QueryPlan; /// Information on what to do when executing a subquery in the [GLOBAL] IN/JOIN section. struct SubqueryForSet { /// The source is obtained using the InterpreterSelectQuery subquery. - Pipe source; + std::unique_ptr source; /// If set, build it from result. SetPtr set; diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 0b654d0f325..c7ffe8b0c9b 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -204,7 +204,7 @@ void QueryPipeline::addCreatingSetsTransform(SubqueriesForSets subqueries_for_se for (auto & subquery : subqueries_for_sets) { - if (!subquery.second.source.empty()) + if (subquery.second.source) { auto & source = sources.emplace_back(std::move(subquery.second.source)); if (source.numOutputPorts() > 1) @@ -315,6 +315,20 @@ QueryPipeline QueryPipeline::unitePipelines( return pipeline; } +void QueryPipeline::addDelayedPipeline(QueryPipeline pipeline) +{ + pipeline.resize(1); + + auto * collected_processors = pipe.collected_processors; + + Pipes pipes; + pipes.emplace_back(QueryPipeline::getPipe(std::move(pipeline))); + pipes.emplace_back(std::move(pipe)); + pipe = Pipe::unitePipes(std::move(pipes), collected_processors); + + pipe.addTransform(std::make_shared(getHeader(), 2)); +} + void QueryPipeline::setProgressCallback(const ProgressCallback & callback) { for (auto & processor : pipe.processors) diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 45b410ab323..06c67c897be 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -87,6 +87,8 @@ public: size_t max_threads_limit = 0, Processors * collected_processors = nullptr); + void addDelayedPipeline(QueryPipeline); + PipelineExecutorPtr execute(); size_t getNumStreams() const { return pipe.numOutputPorts(); } diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 7e840e1531b..9a4c11f9222 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -22,37 +22,91 @@ static ITransformingStep::Traits getTraits() }; } -CreatingSetsStep::CreatingSetsStep( +CreatingSetStep::CreatingSetStep( const DataStream & input_stream_, - SubqueriesForSets subqueries_for_sets_, + Block header, + String description_, + SubqueryForSet subquery_for_set_, SizeLimits network_transfer_limits_, const Context & context_) - : ITransformingStep(input_stream_, input_stream_.header, getTraits()) - , subqueries_for_sets(std::move(subqueries_for_sets_)) + : ITransformingStep(input_stream_, header, getTraits()) + , description(std::move(description_)) + , subquery_for_set(std::move(subquery_for_set_)) , network_transfer_limits(std::move(network_transfer_limits_)) , context(context_) { } -void CreatingSetsStep::transformPipeline(QueryPipeline & pipeline) +void CreatingSetStep::transformPipeline(QueryPipeline & pipeline) { - pipeline.addCreatingSetsTransform(std::move(subqueries_for_sets), network_transfer_limits, context); + pipeline.resize(1); + pipeline.addTransform( + std::make_shared( + pipeline.getHeader(), + getOutputStream().header, + std::move(subquery_for_set), + network_transfer_limits, + context)); } -void CreatingSetsStep::describeActions(FormatSettings & settings) const +void CreatingSetStep::describeActions(FormatSettings & settings) const { String prefix(settings.offset, ' '); - for (const auto & set : subqueries_for_sets) + settings.out << prefix; + if (subquery_for_set.set) + settings.out << "Set: "; + else if (subquery_for_set.join) + settings.out << "Join: "; + + settings.out << description << '\n'; +} + +CreatingSetsStep::CreatingSetsStep(DataStreams input_streams_) +{ + if (input_streams_.empty()) + throw Exception("CreatingSetsStep cannot be created with no inputs", ErrorCodes::LOGICAL_ERROR); + + input_streams = std::move(input_streams_); + output_stream = input_streams.front(); + + for (size_t i = 1; i < input_streams.size(); ++i) + assertBlocksHaveEqualStructure(output_stream->header, input_streams[i].header, "CreatingSets"); +} + +QueryPipelinePtr CreatingSetsStep::updatePipeline(QueryPipelines pipelines) +{ + if (pipelines.empty()) + throw Exception("CreatingSetsStep cannot be created with no inputs", ErrorCodes::LOGICAL_ERROR); + + auto main_pipeline = std::move(pipelines.front()); + if (pipelines.size() == 1) + return main_pipeline; + + std::swap(pipelines.front(), pipelines.back()); + pipelines.pop_back(); + + QueryPipeline delayed_pipeline; + if (pipelines.size() > 1) { - settings.out << prefix; - if (set.second.set) - settings.out << "Set: "; - else if (set.second.join) - settings.out << "Join: "; - - settings.out << set.first << '\n'; + QueryPipelineProcessorsCollector collector(delayed_pipeline, this); + delayed_pipeline = QueryPipeline::unitePipelines(std::move(pipelines), output_stream->header); + processors = collector.detachProcessors(); } + else + delayed_pipeline = std::move(*pipelines.front()); + + QueryPipelineProcessorsCollector collector(*main_pipeline, this); + main_pipeline->addDelayedPipeline(std::move(delayed_pipeline)); + auto added_processors = collector.detachProcessors(); + processors.insert(processors.end(), added_processors.begin(), added_processors.end()); + + return main_pipeline; +} + +void CreatingSetsStep::describePipeline(FormatSettings & settings) const +{ + IQueryPlanStep::describePipeline(processors, settings); } } diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 4ba4863c043..f2351c91518 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -7,12 +7,14 @@ namespace DB { /// Creates sets for subqueries and JOIN. See CreatingSetsTransform. -class CreatingSetsStep : public ITransformingStep +class CreatingSetStep : public ITransformingStep { public: - CreatingSetsStep( + CreatingSetStep( const DataStream & input_stream_, - SubqueriesForSets subqueries_for_sets_, + Block header, + String description_, + SubqueryForSet subquery_for_set_, SizeLimits network_transfer_limits_, const Context & context_); @@ -23,9 +25,25 @@ public: void describeActions(FormatSettings & settings) const override; private: - SubqueriesForSets subqueries_for_sets; + String description; + SubqueryForSet subquery_for_set; SizeLimits network_transfer_limits; const Context & context; }; +class CreatingSetsStep : public IQueryPlanStep +{ +public: + CreatingSetsStep(DataStreams input_streams_); + + String getName() const override { return "CreatingSets"; } + + QueryPipelinePtr updatePipeline(QueryPipelines pipelines) override; + + void describePipeline(FormatSettings & settings) const override; + +private: + Processors processors; +}; + } From 18bb5f026ae4c453c7e293828e047867e29c50bd Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Tue, 15 Sep 2020 16:37:12 +0300 Subject: [PATCH 092/273] changes after review --- docs/ru/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index da1c56e3daf..4854e39d96d 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1618,7 +1618,7 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; ## low\_cardinality\_max\_dictionary\_size {#low_cardinality_max_dictionary_size} -Задает максимальную длину строк в общем глобальном словаре для типа данных `LowCardinality`, который может быть записан в файловую систему хранилища. Настройка предоствращает проблемы с оперативной памятью в случае неограниченного увеличения словаря. Все данные, которые не могут быть закодированы из-за ограничения максимального размера словаря, ClickHouse записывает обычным способом. +Задает максимальный размер общего глобального словаря (в строках) для типа данных `LowCardinality`, который может быть записан в файловую систему хранилища. Настройка предотвращает проблемы с оперативной памятью в случае неограниченного увеличения словаря. Все данные, которые не могут быть закодированы из-за ограничения максимального размера словаря, ClickHouse записывает обычным способом. Допустимые значения: @@ -1628,7 +1628,7 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; ## low\_cardinality\_use\_single\_dictionary\_for\_part {#low_cardinality_use_single_dictionary_for_part} -Включает или выключает использование единого словаря для частей данных. +Включает или выключает использование единого словаря для куска (парта). По умолчанию сервер ClickHouse следит за размером словарей, и если словарь переполняется, сервер создает следующий. Чтобы запретить создание нескольких словарей, задайте настройку `low_cardinality_use_single_dictionary_for_part = 1`. From 7c0cafab0a439cd7e2164c33ac3d1a756cc21db7 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Tue, 15 Sep 2020 16:58:39 +0300 Subject: [PATCH 093/273] resolving conflict --- .../functions/type-conversion-functions.md | 91 +++++++++++++++++++ 1 file changed, 91 insertions(+) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 3b70f0d6577..1c09e4749f6 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -513,6 +513,97 @@ SELECT parseDateTimeBestEffort('10 20:19') - [toDate](#todate) - [toDateTime](#todatetime) +## toUnixTimestamp64Milli +## toUnixTimestamp64Micro +## toUnixTimestamp64Nano + +Преобразует значение `DateTime64` в значение `Int64` с фиксированной точностью менее одной секунды. +Входное значение округляется соответствующим образом вверх или вниз в зависимости от его точности. Обратите внимание, что возвращаемое значение - это временная метка в UTC, а не в часовом поясе `DateTime64`. + +**Синтаксис** + +``` sql +toUnixTimestamp64Milli(value) +``` + +**Параметры** + +- `value` — значение `DateTime64` с любой точностью. + +**Возвращаемое значение** + +- Значение `value`, преобразованное в тип данных `Int64`. + +**Примеры** + +Запрос: + +``` sql +WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 +SELECT toUnixTimestamp64Milli(dt64) +``` + +Ответ: + +``` text +┌─toUnixTimestamp64Milli(dt64)─┐ +│ 1568650812345 │ +└──────────────────────────────┘ +``` + +Запрос: + +``` sql +WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 +SELECT toUnixTimestamp64Nano(dt64) +``` + +Ответ: + +``` text +┌─toUnixTimestamp64Nano(dt64)─┐ +│ 1568650812345678000 │ +└─────────────────────────────┘ +``` + +## fromUnixTimestamp64Milli +## fromUnixTimestamp64Micro +## fromUnixTimestamp64Nano + +Преобразует значение `Int64` в значение `DateTime64` с фиксированной точностью менее одной секунды и дополнительным часовым поясом. Входное значение округляется соответствующим образом вверх или вниз в зависимости от его точности. Обратите внимание, что входное значение обрабатывается как метка времени UTC, а не метка времени в заданном (или неявном) часовом поясе. + +**Синтаксис** + +``` sql +fromUnixTimestamp64Milli(value [, ti]) +``` + +**Параметры** + +- `value` — значение типы `Int64` с любой точностью. +- `timezone` — (не обязательный параметр) часовой пояс в формате `String` для возвращаемого результата. + +**Возвращаемое значение** + +- Значение `value`, преобразованное в тип данных `DateTime64`. + +**Пример** + +Запрос: + +``` sql +WITH CAST(1234567891011, 'Int64') AS i64 +SELECT fromUnixTimestamp64Milli(i64, 'UTC') +``` + +Ответ: + +``` text +┌─fromUnixTimestamp64Milli(i64, 'UTC')─┐ +│ 2009-02-13 23:31:31.011 │ +└──────────────────────────────────────┘ +``` + ## toLowCardinality {#tolowcardinality} Преобразует входные данные в версию [LowCardianlity](../data-types/lowcardinality.md) того же типа данных. From 84c68947b947a382b5424a4f6713b34b12072aec Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 15 Sep 2020 22:07:03 +0800 Subject: [PATCH 094/273] Add test for MySQLBinlogReadBuffer --- .../gtest_mysql_binlog_event_read_buffer.cpp | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/src/IO/tests/gtest_mysql_binlog_event_read_buffer.cpp b/src/IO/tests/gtest_mysql_binlog_event_read_buffer.cpp index 183da5182af..fe16a13085e 100644 --- a/src/IO/tests/gtest_mysql_binlog_event_read_buffer.cpp +++ b/src/IO/tests/gtest_mysql_binlog_event_read_buffer.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -17,4 +18,41 @@ TEST(MySQLBinlogEventReadBuffer, CheckBoundary) } } +TEST(MySQLBinlogEventReadBuffer, NiceBufferSize) +{ + char res[2]; + std::vector memory_data(6, 0x01); + ReadBufferFromMemory nested_in(memory_data.data(), 6); + + MySQLBinlogEventReadBuffer binlog_in(nested_in); + binlog_in.readStrict(res, 2); + ASSERT_EQ(res[0], 0x01); + ASSERT_EQ(res[1], 0x01); + ASSERT_TRUE(binlog_in.eof()); +} + +TEST(MySQLBinlogEventReadBuffer, BadBufferSizes) +{ + char res[4]; + std::vector buffers; + std::vector nested_buffers; + std::vector>> memory_buffers_data; + std::vector bad_buffers_size = {2, 1, 2, 3}; + + for (const auto & bad_buffer_size : bad_buffers_size) + { + memory_buffers_data.emplace_back(std::make_shared>(bad_buffer_size, 0x01)); + buffers.emplace_back(std::make_shared(memory_buffers_data.back()->data(), bad_buffer_size)); + nested_buffers.emplace_back(buffers.back().get()); + } + + ConcatReadBuffer concat_buffer(nested_buffers); + MySQLBinlogEventReadBuffer binlog_in(concat_buffer); + binlog_in.readStrict(res, 4); + ASSERT_EQ(res[0], 0x01); + ASSERT_EQ(res[1], 0x01); + ASSERT_EQ(res[2], 0x01); + ASSERT_EQ(res[3], 0x01); + ASSERT_TRUE(binlog_in.eof()); +} From fb92c56beb688b6cfb0bed91ee33f5e5fae9e930 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 15 Sep 2020 22:22:23 +0800 Subject: [PATCH 095/273] Fix bad and nice buffer size --- src/IO/MySQLBinlogEventReadBuffer.cpp | 14 ++++---- .../gtest_mysql_binlog_event_read_buffer.cpp | 33 ++++++++++++++++--- 2 files changed, 36 insertions(+), 11 deletions(-) diff --git a/src/IO/MySQLBinlogEventReadBuffer.cpp b/src/IO/MySQLBinlogEventReadBuffer.cpp index 3b76efa64ac..2fd8fdca171 100644 --- a/src/IO/MySQLBinlogEventReadBuffer.cpp +++ b/src/IO/MySQLBinlogEventReadBuffer.cpp @@ -22,15 +22,15 @@ bool MySQLBinlogEventReadBuffer::nextImpl() if (in.eof()) return false; - if (likely(in.available() > CHECKSUM_CRC32_SIGNATURE_LENGTH)) - { - working_buffer = ReadBuffer::Buffer(in.position(), in.buffer().end() - CHECKSUM_CRC32_SIGNATURE_LENGTH); - in.ignore(working_buffer.size()); - return true; - } - if (checksum_buff_size == checksum_buff_limit) { + if (likely(in.available() > CHECKSUM_CRC32_SIGNATURE_LENGTH)) + { + working_buffer = ReadBuffer::Buffer(in.position(), in.buffer().end() - CHECKSUM_CRC32_SIGNATURE_LENGTH); + in.ignore(working_buffer.size()); + return true; + } + in.readStrict(checksum_buf, CHECKSUM_CRC32_SIGNATURE_LENGTH); checksum_buff_size = checksum_buff_limit = CHECKSUM_CRC32_SIGNATURE_LENGTH; } diff --git a/src/IO/tests/gtest_mysql_binlog_event_read_buffer.cpp b/src/IO/tests/gtest_mysql_binlog_event_read_buffer.cpp index fe16a13085e..66077bcba5b 100644 --- a/src/IO/tests/gtest_mysql_binlog_event_read_buffer.cpp +++ b/src/IO/tests/gtest_mysql_binlog_event_read_buffer.cpp @@ -49,10 +49,35 @@ TEST(MySQLBinlogEventReadBuffer, BadBufferSizes) ConcatReadBuffer concat_buffer(nested_buffers); MySQLBinlogEventReadBuffer binlog_in(concat_buffer); binlog_in.readStrict(res, 4); - ASSERT_EQ(res[0], 0x01); - ASSERT_EQ(res[1], 0x01); - ASSERT_EQ(res[2], 0x01); - ASSERT_EQ(res[3], 0x01); + + for (size_t index = 0; index < 4; ++index) + ASSERT_EQ(res[index], 0x01); + + ASSERT_TRUE(binlog_in.eof()); +} + +TEST(MySQLBinlogEventReadBuffer, NiceAndBadBufferSizes) +{ + char res[12]; + std::vector buffers; + std::vector nested_buffers; + std::vector>> memory_buffers_data; + std::vector buffers_size = {6, 1, 3, 6}; + + for (const auto & bad_buffer_size : buffers_size) + { + memory_buffers_data.emplace_back(std::make_shared>(bad_buffer_size, 0x01)); + buffers.emplace_back(std::make_shared(memory_buffers_data.back()->data(), bad_buffer_size)); + nested_buffers.emplace_back(buffers.back().get()); + } + + ConcatReadBuffer concat_buffer(nested_buffers); + MySQLBinlogEventReadBuffer binlog_in(concat_buffer); + binlog_in.readStrict(res, 12); + + for (size_t index = 0; index < 12; ++index) + ASSERT_EQ(res[index], 0x01); + ASSERT_TRUE(binlog_in.eof()); } From 7465e00163a5e02fa6928513a6cae89023dcab5d Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Tue, 15 Sep 2020 17:22:32 +0300 Subject: [PATCH 096/273] Optimized marks selection algorithm for continuous marks ranges --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 94 +++++++------------ 1 file changed, 35 insertions(+), 59 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index e780ebda111..f2010b4e34e 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1498,79 +1498,55 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( } else { - // Do inclusion search, where we only look for one range + // For the case of one continuous range of keys we use binary search algorithm + + LOG_TRACE(log, "Running binary search on index range for part {} ({} marks)", part->name, marks_count); size_t steps = 0; - auto find_leaf = [&](bool left) -> std::optional + MarkRange result_range; + + size_t searched_left = 0; + size_t searched_right = marks_count; + + while (searched_left + 1 < searched_right) { - std::vector stack = {}; - - MarkRange range = {0, marks_count}; - - steps++; - + const size_t middle = (searched_left + searched_right) / 2; + MarkRange range(0, middle); if (may_be_true_in_range(range)) - stack.emplace_back(range.begin, range.end); + searched_right = middle; + else + searched_left = middle; + ++steps; + } + result_range.begin = searched_left; + LOG_TRACE(log, "Found (LEFT) boundary mark: {}", searched_left); - while (!stack.empty()) - { - range = stack.back(); - stack.pop_back(); + searched_right = marks_count; + while (searched_left + 1 < searched_right) + { + const size_t middle = (searched_left + searched_right) / 2; + MarkRange range(middle, marks_count); + if (may_be_true_in_range(range)) + searched_left = middle; + else + searched_right = middle; + ++steps; + } + result_range.end = searched_right; + LOG_TRACE(log, "Found (RIGHT) boundary mark: {}", searched_right); - if (range.end == range.begin + 1) - { - if (left) - return range.begin; - else - return range.end; - } - else - { - std::vector check_order = {}; - MarkRange left_range = {range.begin, (range.begin + range.end) / 2}; - MarkRange right_range = {(range.begin + range.end) / 2, range.end}; + if (may_be_true_in_range(result_range)) + res.emplace_back(std::move(result_range)); - if (left) - { - check_order.emplace_back(left_range.begin, left_range.end); - check_order.emplace_back(right_range.begin, right_range.end); - } - else - { - check_order.emplace_back(right_range.begin, right_range.end); - check_order.emplace_back(left_range.begin, left_range.end); - } - - steps++; - - if (may_be_true_in_range(check_order[0])) - { - stack.emplace_back(check_order[0].begin, check_order[0].end); - continue; - } - - if (may_be_true_in_range(check_order[1])) - stack.emplace_back(check_order[1].begin, check_order[1].end); - else - break; // No mark range would suffice - } - } - - return std::nullopt; - }; - - auto left_leaf = find_leaf(true); - if (left_leaf) - res.emplace_back(left_leaf.value(), find_leaf(false).value()); - - LOG_TRACE(log, "Used optimized inclusion search over index for part {} with {} steps", part->name, steps); + LOG_TRACE(log, "Found {} range in {} steps", res.empty() ? "empty" : "continuous", steps); } return res; } + MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( MergeTreeIndexPtr index_helper, MergeTreeIndexConditionPtr condition, From 9c329996ffbf1f21c55dccc970d58d41ed50d6b8 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 15 Sep 2020 22:56:57 +0800 Subject: [PATCH 097/273] Revert param name in header file --- src/Core/MySQL/MySQLReplication.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index 12ed67633b0..ad5e53ed200 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -505,7 +505,7 @@ namespace MySQLReplication class MySQLFlavor : public IFlavor { public: - void readPayloadImpl(ReadBuffer & event_payload) override; + void readPayloadImpl(ReadBuffer & payload) override; String getName() const override { return "MySQL"; } Position getPosition() const override { return position; } BinlogEventPtr readOneEvent() override { return event; } From 30352f096dcae6792320a620e957ee6da333dcd2 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 15 Sep 2020 22:59:21 +0800 Subject: [PATCH 098/273] Init buffer in constructor --- src/IO/MySQLBinlogEventReadBuffer.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/IO/MySQLBinlogEventReadBuffer.cpp b/src/IO/MySQLBinlogEventReadBuffer.cpp index 2fd8fdca171..0680e075af9 100644 --- a/src/IO/MySQLBinlogEventReadBuffer.cpp +++ b/src/IO/MySQLBinlogEventReadBuffer.cpp @@ -12,6 +12,7 @@ namespace ErrorCodes MySQLBinlogEventReadBuffer::MySQLBinlogEventReadBuffer(ReadBuffer & in_) : ReadBuffer(nullptr, 0, 0), in(in_) { + nextIfAtEnd(); } bool MySQLBinlogEventReadBuffer::nextImpl() From a792850ecd69934e4294d7b65ba1a14459e9de1f Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Tue, 15 Sep 2020 18:05:42 +0300 Subject: [PATCH 099/273] Update ThreadPool.cpp --- src/Common/ThreadPool.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 93aa6be8d9a..737826e3027 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -279,7 +279,7 @@ void GlobalThreadPool::initialize(size_t max_threads) { if (the_instance) { - throw Exception(LOGICAL_ERROR, + throw Exception(ErrorCodes::LOGICAL_ERROR, "The global thread pool is initialized twice"); } From 0c06ccc35ea9df6fd0859ec5bec5f5a51bcdf0f7 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 15 Sep 2020 23:26:13 +0800 Subject: [PATCH 100/273] Fix parse error packet on event --- src/Core/MySQL/MySQLReplication.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 81d46d10025..a46d787b225 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -803,20 +803,20 @@ namespace MySQLReplication void MySQLFlavor::readPayloadImpl(ReadBuffer & payload) { - MySQLBinlogEventReadBuffer event_payload(payload); - UInt16 header = static_cast(*event_payload.position()); + UInt16 header = static_cast(*payload.position()); switch (header) { case PACKET_EOF: throw ReplicationError("Master maybe lost", ErrorCodes::UNKNOWN_EXCEPTION); case PACKET_ERR: ERRPacket err; - err.readPayloadWithUnpacked(event_payload); + err.readPayloadWithUnpacked(payload); throw ReplicationError(err.error_message, ErrorCodes::UNKNOWN_EXCEPTION); } // skip the header flag. - event_payload.ignore(1); + payload.ignore(1); + MySQLBinlogEventReadBuffer event_payload(payload); EventType event_type = static_cast(*(event_payload.position() + 4)); switch (event_type) { From 5afb19faf1893113e978a330c42418a0cc0f3fba Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Tue, 15 Sep 2020 19:58:09 +0300 Subject: [PATCH 101/273] Update ThreadPool.cpp --- src/Common/ThreadPool.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 737826e3027..cb8a7669eef 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -279,7 +279,7 @@ void GlobalThreadPool::initialize(size_t max_threads) { if (the_instance) { - throw Exception(ErrorCodes::LOGICAL_ERROR, + throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "The global thread pool is initialized twice"); } From 118a8a513e6eea1b24d5f5e4464c0e746c597046 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 15 Sep 2020 20:13:13 +0300 Subject: [PATCH 102/273] Update mutations interpreter. --- src/Interpreters/InterpreterSelectQuery.cpp | 29 +--------- .../InterpreterSelectWithUnionQuery.cpp | 7 ++- src/Interpreters/MutationsInterpreter.cpp | 50 ++++++++++------- src/Interpreters/MutationsInterpreter.h | 5 +- src/Interpreters/SubqueryForSet.cpp | 5 ++ src/Interpreters/SubqueryForSet.h | 5 ++ src/Processors/QueryPipeline.cpp | 55 +------------------ src/Processors/QueryPipeline.h | 6 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 37 ++++++++++++- src/Processors/QueryPlan/CreatingSetsStep.h | 8 ++- src/Processors/QueryPlan/QueryPlan.cpp | 14 +++-- src/Processors/QueryPlan/QueryPlan.h | 4 +- .../QueryPlan/ReadFromPreparedSource.cpp | 3 +- .../QueryPlan/ReadFromPreparedSource.h | 2 +- src/Processors/Transforms/JoiningTransform.h | 2 +- 15 files changed, 111 insertions(+), 121 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6f1be43b3a1..6d4362baeca 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1866,34 +1866,7 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_p return; SizeLimits limits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode); - - std::vector plans; - DataStreams input_streams; - input_streams.emplace_back(query_plan.getCurrentDataStream()); - - for (auto & [description, set] : subqueries_for_sets) - { - auto plan = std::move(set.source); - std::string type = (set.join != nullptr) ? "JOIN" - : "subquery"; - - auto creating_set = std::make_unique( - plan->getCurrentDataStream(), - query_plan.getCurrentDataStream().header, - std::move(description), - std::move(set), - limits, - *context); - creating_set->setStepDescription("Create set for " + type); - plan->addStep(std::move(creating_set)); - - input_streams.emplace_back(plan->getCurrentDataStream()); - plans.emplace_back(std::move(*plan)); - } - - auto creating_sets = std::make_unique(std::move(input_streams)); - creating_sets->setStepDescription("Create sets before main query execution"); - query_plan.unitePlans(std::move(creating_sets), std::move(plans)); + addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), limits, *context); } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 1e631ea538b..ba0ebfaaf27 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -183,13 +183,14 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) return; } - std::vector plans(num_plans); + std::vector> plans(num_plans); DataStreams data_streams(num_plans); for (size_t i = 0; i < num_plans; ++i) { - nested_interpreters[i]->buildQueryPlan(plans[i]); - data_streams[i] = plans[i].getCurrentDataStream(); + plans[i] = std::make_unique(); + nested_interpreters[i]->buildQueryPlan(*plans[i]); + data_streams[i] = plans[i]->getCurrentDataStream(); } auto max_threads = context->getSettingsRef().max_threads; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 089e3d1c23f..7f6f3441f5b 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -11,6 +11,11 @@ #include #include #include +#include +#include +#include +#include +#include #include #include #include @@ -19,6 +24,7 @@ #include #include #include +#include namespace DB @@ -524,10 +530,11 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) SelectQueryOptions().analyze(/* dry_run = */ false).ignoreLimits()}; auto first_stage_header = interpreter.getSampleBlock(); - QueryPipeline pipeline; - pipeline.init(Pipe(std::make_shared(first_stage_header))); - addStreamsForLaterStages(stages_copy, pipeline); - updated_header = std::make_unique(pipeline.getHeader()); + QueryPlan plan; + auto source = std::make_shared(first_stage_header); + plan.addStep(std::make_unique(Pipe(std::move(source)))); + auto pipeline = addStreamsForLaterStages(stages_copy, plan); + updated_header = std::make_unique(pipeline->getHeader()); } /// Special step to recalculate affected indices and TTL expressions. @@ -656,7 +663,7 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & return select; } -void MutationsInterpreter::addStreamsForLaterStages(const std::vector & prepared_stages, QueryPipeline & pipeline) const +QueryPipelinePtr MutationsInterpreter::addStreamsForLaterStages(const std::vector & prepared_stages, QueryPlan & plan) const { for (size_t i_stage = 1; i_stage < prepared_stages.size(); ++i_stage) { @@ -668,18 +675,12 @@ void MutationsInterpreter::addStreamsForLaterStages(const std::vector & p if (i < stage.filter_column_names.size()) { /// Execute DELETEs. - pipeline.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, step->actions(), stage.filter_column_names[i], false); - }); + plan.addStep(std::make_unique(plan.getCurrentDataStream(), step->actions(), stage.filter_column_names[i], false)); } else { /// Execute UPDATE or final projection. - pipeline.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, step->actions()); - }); + plan.addStep(std::make_unique(plan.getCurrentDataStream(), step->actions())); } } @@ -689,14 +690,17 @@ void MutationsInterpreter::addStreamsForLaterStages(const std::vector & p const Settings & settings = context.getSettingsRef(); SizeLimits network_transfer_limits( settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode); - pipeline.addCreatingSetsTransform(std::move(subqueries_for_sets), network_transfer_limits, context); + addCreatingSetsStep(plan, std::move(subqueries_for_sets), network_transfer_limits, context); } } - pipeline.addSimpleTransform([&](const Block & header) + auto pipeline = plan.buildQueryPipeline(); + pipeline->addSimpleTransform([&](const Block & header) { return std::make_shared(header); }); + + return pipeline; } void MutationsInterpreter::validate() @@ -718,8 +722,11 @@ void MutationsInterpreter::validate() } } - auto block_io = select_interpreter->execute(); - addStreamsForLaterStages(stages, block_io.pipeline); + QueryPlan plan; + select_interpreter->buildQueryPlan(plan); + addStreamsForLaterStages(stages, plan); + + auto pipeline = plan.buildQueryPipeline(); } BlockInputStreamPtr MutationsInterpreter::execute() @@ -727,10 +734,13 @@ BlockInputStreamPtr MutationsInterpreter::execute() if (!can_execute) throw Exception("Cannot execute mutations interpreter because can_execute flag set to false", ErrorCodes::LOGICAL_ERROR); - auto block_io = select_interpreter->execute(); - addStreamsForLaterStages(stages, block_io.pipeline); + QueryPlan plan; + select_interpreter->buildQueryPlan(plan); - auto result_stream = block_io.getInputStream(); + addStreamsForLaterStages(stages, plan); + + auto pipeline = plan.buildQueryPipeline(); + BlockInputStreamPtr result_stream = std::make_shared(std::move(*pipeline)); /// Sometimes we update just part of columns (for example UPDATE mutation) /// in this case we don't read sorting key, so just we don't check anything. diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 359ee1a3fd0..59d9e7657c3 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -13,7 +13,10 @@ namespace DB { class Context; +class QueryPlan; + class QueryPipeline; +using QueryPipelinePtr = std::unique_ptr; /// Return false if the data isn't going to be changed by mutations. bool isStorageTouchedByMutations( @@ -52,7 +55,7 @@ private: struct Stage; ASTPtr prepareInterpreterSelectQuery(std::vector &prepared_stages, bool dry_run); - void addStreamsForLaterStages(const std::vector & prepared_stages, QueryPipeline & pipeline) const; + QueryPipelinePtr addStreamsForLaterStages(const std::vector & prepared_stages, QueryPlan & plan) const; std::optional getStorageSortDescriptionIfPossible(const Block & header) const; diff --git a/src/Interpreters/SubqueryForSet.cpp b/src/Interpreters/SubqueryForSet.cpp index e944b76e71c..17ea813c545 100644 --- a/src/Interpreters/SubqueryForSet.cpp +++ b/src/Interpreters/SubqueryForSet.cpp @@ -8,6 +8,11 @@ namespace DB { +SubqueryForSet::SubqueryForSet() = default; +SubqueryForSet::~SubqueryForSet() = default; +SubqueryForSet::SubqueryForSet(SubqueryForSet &&) = default; +SubqueryForSet & SubqueryForSet::operator= (SubqueryForSet &&) = default; + void SubqueryForSet::makeSource(std::shared_ptr & interpreter, NamesWithAliases && joined_block_aliases_) { diff --git a/src/Interpreters/SubqueryForSet.h b/src/Interpreters/SubqueryForSet.h index b44f0f6cf8b..fd073500dc2 100644 --- a/src/Interpreters/SubqueryForSet.h +++ b/src/Interpreters/SubqueryForSet.h @@ -18,6 +18,11 @@ class QueryPlan; /// Information on what to do when executing a subquery in the [GLOBAL] IN/JOIN section. struct SubqueryForSet { + SubqueryForSet(); + ~SubqueryForSet(); + SubqueryForSet(SubqueryForSet &&); + SubqueryForSet & operator= (SubqueryForSet &&); + /// The source is obtained using the InterpreterSelectQuery subquery. std::unique_ptr source; diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index c7ffe8b0c9b..0337f19c2e3 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -196,59 +196,6 @@ void QueryPipeline::addExtremesTransform() pipe.addTransform(std::move(transform), nullptr, port); } -void QueryPipeline::addCreatingSetsTransform(SubqueriesForSets subqueries_for_sets, const SizeLimits & network_transfer_limits, const Context & context) -{ - checkInitializedAndNotCompleted(); - - Pipes sources; - - for (auto & subquery : subqueries_for_sets) - { - if (subquery.second.source) - { - auto & source = sources.emplace_back(std::move(subquery.second.source)); - if (source.numOutputPorts() > 1) - source.addTransform(std::make_shared(source.getHeader(), source.numOutputPorts(), 1)); - - source.dropExtremes(); - - auto creating_sets = std::make_shared( - source.getHeader(), - getHeader(), - std::move(subquery.second), - network_transfer_limits, - context); - - InputPort * totals = nullptr; - if (source.getTotalsPort()) - totals = creating_sets->addTotalsPort(); - - source.addTransform(std::move(creating_sets), totals, nullptr); - } - } - - if (sources.empty()) - return; - - auto * collected_processors = pipe.collected_processors; - - /// We unite all sources together. - /// Set collected_processors to attach all newly-added processors to current query plan step. - auto source = Pipe::unitePipes(std::move(sources), collected_processors); - if (source.numOutputPorts() > 1) - source.addTransform(std::make_shared(source.getHeader(), source.numOutputPorts(), 1)); - source.collected_processors = nullptr; - - resize(1); - - Pipes pipes; - pipes.emplace_back(std::move(source)); - pipes.emplace_back(std::move(pipe)); - pipe = Pipe::unitePipes(std::move(pipes), collected_processors); - - pipe.addTransform(std::make_shared(getHeader(), 2)); -} - void QueryPipeline::setOutputFormat(ProcessorPtr output) { checkInitializedAndNotCompleted(); @@ -315,7 +262,7 @@ QueryPipeline QueryPipeline::unitePipelines( return pipeline; } -void QueryPipeline::addDelayedPipeline(QueryPipeline pipeline) +void QueryPipeline::addDelayingPipeline(QueryPipeline pipeline) { pipeline.resize(1); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 06c67c897be..4be7bbfd573 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -55,8 +55,6 @@ public: void addTotalsHavingTransform(ProcessorPtr transform); /// Add transform which calculates extremes. This transform adds extremes port and doesn't change inputs number. void addExtremesTransform(); - /// Adds transform which creates sets. It will be executed before reading any data from input ports. - void addCreatingSetsTransform(SubqueriesForSets subqueries_for_sets, const SizeLimits & network_transfer_limits, const Context & context); /// Resize pipeline to single output and add IOutputFormat. Pipeline will be completed after this transformation. void setOutputFormat(ProcessorPtr output); /// Get current OutputFormat. @@ -87,7 +85,9 @@ public: size_t max_threads_limit = 0, Processors * collected_processors = nullptr); - void addDelayedPipeline(QueryPipeline); + /// Add other pipeline and execute it before current one. + /// Pipeline must have same header. + void addDelayingPipeline(QueryPipeline pipeline); PipelineExecutorPtr execute(); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 9a4c11f9222..f63b657c029 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -97,7 +97,7 @@ QueryPipelinePtr CreatingSetsStep::updatePipeline(QueryPipelines pipelines) delayed_pipeline = std::move(*pipelines.front()); QueryPipelineProcessorsCollector collector(*main_pipeline, this); - main_pipeline->addDelayedPipeline(std::move(delayed_pipeline)); + main_pipeline->addDelayingPipeline(std::move(delayed_pipeline)); auto added_processors = collector.detachProcessors(); processors.insert(processors.end(), added_processors.begin(), added_processors.end()); @@ -109,4 +109,39 @@ void CreatingSetsStep::describePipeline(FormatSettings & settings) const IQueryPlanStep::describePipeline(processors, settings); } +void addCreatingSetsStep( + QueryPlan & query_plan, SubqueriesForSets subqueries_for_sets, const SizeLimits & limits, const Context & context) +{ + DataStreams input_streams; + input_streams.emplace_back(query_plan.getCurrentDataStream()); + + std::vector> plans; + plans.emplace_back(std::make_unique(std::move(query_plan))); + query_plan = QueryPlan(); + + for (auto & [description, set] : subqueries_for_sets) + { + auto plan = std::move(set.source); + std::string type = (set.join != nullptr) ? "JOIN" + : "subquery"; + + auto creating_set = std::make_unique( + plan->getCurrentDataStream(), + input_streams.front().header, + std::move(description), + std::move(set), + limits, + context); + creating_set->setStepDescription("Create set for " + type); + plan->addStep(std::move(creating_set)); + + input_streams.emplace_back(plan->getCurrentDataStream()); + plans.emplace_back(std::move(plan)); + } + + auto creating_sets = std::make_unique(std::move(input_streams)); + creating_sets->setStepDescription("Create sets before main query execution"); + query_plan.unitePlans(std::move(creating_sets), std::move(plans)); +} + } diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index f2351c91518..ec13ab2052e 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -18,7 +18,7 @@ public: SizeLimits network_transfer_limits_, const Context & context_); - String getName() const override { return "CreatingSets"; } + String getName() const override { return "CreatingSet"; } void transformPipeline(QueryPipeline & pipeline) override; @@ -46,4 +46,10 @@ private: Processors processors; }; +void addCreatingSetsStep( + QueryPlan & query_plan, + SubqueriesForSets subqueries_for_sets, + const SizeLimits & limits, + const Context & context); + } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 31b9de2fcee..74781f4b5d9 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -26,6 +26,8 @@ namespace ErrorCodes QueryPlan::QueryPlan() = default; QueryPlan::~QueryPlan() = default; +QueryPlan::QueryPlan(QueryPlan &&) = default; +QueryPlan & QueryPlan::operator=(QueryPlan &&) = default; void QueryPlan::checkInitialized() const { @@ -51,7 +53,7 @@ const DataStream & QueryPlan::getCurrentDataStream() const return root->step->getOutputStream(); } -void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vector plans) +void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vector> plans) { if (isInitialized()) throw Exception("Cannot unite plans because current QueryPlan is already initialized", @@ -70,7 +72,7 @@ void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vector plans) for (size_t i = 0; i < num_inputs; ++i) { const auto & step_header = inputs[i].header; - const auto & plan_header = plans[i].getCurrentDataStream().header; + const auto & plan_header = plans[i]->getCurrentDataStream().header; if (!blocksHaveEqualStructure(step_header, plan_header)) throw Exception("Cannot unite QueryPlans using " + step->getName() + " because " "it has incompatible header with plan " + root->step->getName() + " " @@ -79,19 +81,19 @@ void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vector plans) } for (auto & plan : plans) - nodes.splice(nodes.end(), std::move(plan.nodes)); + nodes.splice(nodes.end(), std::move(plan->nodes)); nodes.emplace_back(Node{.step = std::move(step)}); root = &nodes.back(); for (auto & plan : plans) - root->children.emplace_back(plan.root); + root->children.emplace_back(plan->root); for (auto & plan : plans) { - max_threads = std::max(max_threads, plan.max_threads); + max_threads = std::max(max_threads, plan->max_threads); interpreter_context.insert(interpreter_context.end(), - plan.interpreter_context.begin(), plan.interpreter_context.end()); + plan->interpreter_context.begin(), plan->interpreter_context.end()); } } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 7ce8d9426c4..6296eac7502 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -25,8 +25,10 @@ class QueryPlan public: QueryPlan(); ~QueryPlan(); + QueryPlan(QueryPlan &&); + QueryPlan & operator=(QueryPlan &&); - void unitePlans(QueryPlanStepPtr step, std::vector plans); + void unitePlans(QueryPlanStepPtr step, std::vector> plans); void addStep(QueryPlanStepPtr step); bool isInitialized() const { return root != nullptr; } /// Tree is not empty diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp index 6f0d1693ce0..979b4101046 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp @@ -14,7 +14,8 @@ ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, std::shared_ptr context_); + explicit ReadFromPreparedSource(Pipe pipe_, std::shared_ptr context_ = nullptr); String getName() const override { return "ReadNothing"; } diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index c00ac5b83dd..15a203635e2 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -14,7 +14,7 @@ public: JoiningTransform(Block input_header, JoinPtr join_, bool on_totals_ = false, bool default_totals_ = false); - String getName() const override { return "InflatingExpressionTransform"; } + String getName() const override { return "JoiningTransform"; } static Block transformHeader(Block header, const JoinPtr & join); From 13edfeea1fa7c0e9f0a5b840240933b4854a6ee0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 15 Sep 2020 20:23:49 +0300 Subject: [PATCH 103/273] Fix build. --- src/Storages/IStorage.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index dc7c684d5b4..c68b910494e 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -53,6 +53,9 @@ class QueryPlan; class StoragePolicy; using StoragePolicyPtr = std::shared_ptr; +struct StreamLocalLimits; +class EnabledQuota; + struct ColumnSize { size_t marks = 0; From 9100dcd37aa5b873d357888b17def7936cb9311a Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 16 Sep 2020 11:13:17 +0800 Subject: [PATCH 104/273] fix build and test failure --- src/IO/MySQLBinlogEventReadBuffer.cpp | 13 +++++++++++++ src/IO/MySQLBinlogEventReadBuffer.h | 2 ++ .../tests/gtest_mysql_binlog_event_read_buffer.cpp | 11 +++++------ 3 files changed, 20 insertions(+), 6 deletions(-) diff --git a/src/IO/MySQLBinlogEventReadBuffer.cpp b/src/IO/MySQLBinlogEventReadBuffer.cpp index 0680e075af9..9cc7fd4e2ba 100644 --- a/src/IO/MySQLBinlogEventReadBuffer.cpp +++ b/src/IO/MySQLBinlogEventReadBuffer.cpp @@ -59,4 +59,17 @@ bool MySQLBinlogEventReadBuffer::nextImpl() return true; } +MySQLBinlogEventReadBuffer::~MySQLBinlogEventReadBuffer() +{ + try + { + /// ignore last 4 bytes + nextIfAtEnd(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + } diff --git a/src/IO/MySQLBinlogEventReadBuffer.h b/src/IO/MySQLBinlogEventReadBuffer.h index 7a19461e57e..e9452aa551e 100644 --- a/src/IO/MySQLBinlogEventReadBuffer.h +++ b/src/IO/MySQLBinlogEventReadBuffer.h @@ -18,6 +18,8 @@ protected: bool nextImpl() override; public: + ~MySQLBinlogEventReadBuffer() override; + MySQLBinlogEventReadBuffer(ReadBuffer & in_); }; diff --git a/src/IO/tests/gtest_mysql_binlog_event_read_buffer.cpp b/src/IO/tests/gtest_mysql_binlog_event_read_buffer.cpp index 66077bcba5b..f4d39c73a7c 100644 --- a/src/IO/tests/gtest_mysql_binlog_event_read_buffer.cpp +++ b/src/IO/tests/gtest_mysql_binlog_event_read_buffer.cpp @@ -13,8 +13,7 @@ TEST(MySQLBinlogEventReadBuffer, CheckBoundary) std::vector memory_data(index, 0x01); ReadBufferFromMemory nested_in(memory_data.data(), index); - MySQLBinlogEventReadBuffer binlog_in(nested_in); - EXPECT_THROW(binlog_in.ignore(), Exception); + EXPECT_THROW({ MySQLBinlogEventReadBuffer binlog_in(nested_in); }, Exception); } } @@ -50,8 +49,8 @@ TEST(MySQLBinlogEventReadBuffer, BadBufferSizes) MySQLBinlogEventReadBuffer binlog_in(concat_buffer); binlog_in.readStrict(res, 4); - for (size_t index = 0; index < 4; ++index) - ASSERT_EQ(res[index], 0x01); + for (const auto & res_byte : res) + ASSERT_EQ(res_byte, 0x01); ASSERT_TRUE(binlog_in.eof()); } @@ -75,8 +74,8 @@ TEST(MySQLBinlogEventReadBuffer, NiceAndBadBufferSizes) MySQLBinlogEventReadBuffer binlog_in(concat_buffer); binlog_in.readStrict(res, 12); - for (size_t index = 0; index < 12; ++index) - ASSERT_EQ(res[index], 0x01); + for (const auto & res_byte : res) + ASSERT_EQ(res_byte, 0x01); ASSERT_TRUE(binlog_in.eof()); } From 85e990f5184099d107d0957731085a32212f41c7 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 16 Sep 2020 11:30:14 +0800 Subject: [PATCH 105/273] Fix code style --- src/IO/MySQLBinlogEventReadBuffer.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/IO/MySQLBinlogEventReadBuffer.cpp b/src/IO/MySQLBinlogEventReadBuffer.cpp index 9cc7fd4e2ba..3a2aba045d3 100644 --- a/src/IO/MySQLBinlogEventReadBuffer.cpp +++ b/src/IO/MySQLBinlogEventReadBuffer.cpp @@ -4,11 +4,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - MySQLBinlogEventReadBuffer::MySQLBinlogEventReadBuffer(ReadBuffer & in_) : ReadBuffer(nullptr, 0, 0), in(in_) { From c5f3f07983c5ea17c3dca7bda0ade2a732305636 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Wed, 16 Sep 2020 09:13:28 +0300 Subject: [PATCH 106/273] Translated into Russian. --- docs/ru/commercial/index.md | 10 ++++++++ .../mergetree-family/mergetree.md | 5 +--- docs/ru/operations/system-tables/tables.md | 7 ++++-- docs/ru/sql-reference/statements/index.md | 24 +++++++++++++++++++ 4 files changed, 40 insertions(+), 6 deletions(-) diff --git a/docs/ru/commercial/index.md b/docs/ru/commercial/index.md index 6bc3c781161..c6c440c17e8 100644 --- a/docs/ru/commercial/index.md +++ b/docs/ru/commercial/index.md @@ -6,4 +6,14 @@ toc_title: "\u041A\u043E\u043C\u043C\u0435\u0440\u0447\u0435\u0441\u043A\u0438\u \ \u0443\u0441\u043B\u0443\u0433\u0438" --- +# Коммерческие услуги {#clickhouse-commercial-services} +Данный раздел содержит описание коммерческих услуг, предоставляемых для ClickHouse. Поставщики этих услуг — независимые компании, которые могут не быть аффилированы с Яндексом. + +Категории услуг: + +- Облачные услуги [Cloud](../commercial/cloud.md) +- Поддержка [Support](../commercial/support.md) + +!!! note "Для поставщиков услуг" + Если вы — представитель компании-поставщика услуг, вы можете отправить запрос на добавление вашей компании и ваших услуг в соответствующий раздел данной документации (или на добавление нового раздела, если ваши услуги не соответствуют ни одной из существующих категорий). Чтобы отправить запрос (pull-request) на добавление описания в документацию, нажмите на значок "карандаша" в правом верхнем углу страницы. Если ваши услуги доступны в только отдельных регионах, не забудьте указать это на соответствующих локализованных страницах (и обязательно отметьте это при отправке заявки). diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 3c80fe663f1..881c11152cd 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -43,9 +43,6 @@ ORDER BY expr Описание параметров смотрите в [описании запроса CREATE](../../../engines/table-engines/mergetree-family/mergetree.md). -!!! note "Примечание" - `INDEX` — экспериментальная возможность, смотрите [Индексы пропуска данных](#table_engine-mergetree-data_skipping-indexes). - ### Секции запроса {#mergetree-query-clauses} - `ENGINE` — имя и параметры движка. `ENGINE = MergeTree()`. `MergeTree` не имеет параметров. @@ -269,7 +266,7 @@ ClickHouse не может использовать индекс, если зн ClickHouse использует эту логику не только для последовательностей дней месяца, но и для любого частично-монотонного первичного ключа. -### Индексы пропуска данных (экспериментальная функциональность) {#table_engine-mergetree-data_skipping-indexes} +### Индексы пропуска данных {#table_engine-mergetree-data_skipping-indexes} Объявление индексов при определении столбцов в запросе `CREATE`. diff --git a/docs/ru/operations/system-tables/tables.md b/docs/ru/operations/system-tables/tables.md index 7b3ea0037b8..52de10871b2 100644 --- a/docs/ru/operations/system-tables/tables.md +++ b/docs/ru/operations/system-tables/tables.md @@ -24,13 +24,16 @@ - [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) - [Distributed](../../engines/table-engines/special/distributed.md#distributed) -- `total_rows` (Nullable(UInt64)) - Общее количество строк, если есть возможность быстро определить точное количество строк в таблице, в противном случае `Null` (включая базовую таблицу `Buffer`). +- `total_rows` (Nullable(UInt64)) - общее количество строк, если есть возможность быстро определить точное количество строк в таблице, в противном случае `Null` (включая базовую таблицу `Buffer`). -- `total_bytes` (Nullable(UInt64)) - Общее количество байт, если можно быстро определить точное количество байт для таблицы на накопителе, в противном случае `Null` (**не включает** в себя никакого базового хранилища). +- `total_bytes` (Nullable(UInt64)) - общее количество байт, если можно быстро определить точное количество байт для таблицы на накопителе, в противном случае `Null` (**не включает** в себя никакого базового хранилища). - Если таблица хранит данные на диске, возвращает используемое пространство на диске (т. е. сжатое). - Если таблица хранит данные в памяти, возвращает приблизительное количество используемых байт в памяти. +- `lifetime_rows` (Nullable(UInt64)) - общее количество строк, добавленных оператором `INSERT` с момента запуска сервера (только для таблиц `Buffer`). + +- `lifetime_bytes` (Nullable(UInt64)) - общее количество байт, добавленных оператором `INSERT` с момента запуска сервера (только для таблиц `Buffer`). Таблица `system.tables` используется при выполнении запроса `SHOW TABLES`. diff --git a/docs/ru/sql-reference/statements/index.md b/docs/ru/sql-reference/statements/index.md index 4dad718d721..c7862015e64 100644 --- a/docs/ru/sql-reference/statements/index.md +++ b/docs/ru/sql-reference/statements/index.md @@ -3,4 +3,28 @@ toc_folder_title: "\u0412\u044B\u0440\u0430\u0436\u0435\u043D\u0438\u044F" toc_priority: 31 --- +# SQL выражения в ClickHouse {#clickhouse-sql-statements} +Выражения описывают различные действия, которые можно выполнить с помощью SQL запросов. Каждый вид выражения имеет свой синтаксис и особенности использования, которые описаны в соответствующих разделах документации: + +- [SELECT](../../sql-reference/statements/select/index.md) +- [INSERT INTO](../../sql-reference/statements/insert-into.md) +- [CREATE](../../sql-reference/statements/create/index.md) +- [ALTER](../../sql-reference/statements/alter/index.md) +- [SYSTEM](../../sql-reference/statements/system.md) +- [SHOW](../../sql-reference/statements/show.md) +- [GRANT](../../sql-reference/statements/grant.md) +- [REVOKE](../../sql-reference/statements/revoke.md) +- [ATTACH](../../sql-reference/statements/attach.md) +- [CHECK TABLE](../../sql-reference/statements/check-table.md) +- [DESCRIBE TABLE](../../sql-reference/statements/describe-table.md) +- [DETACH](../../sql-reference/statements/detach.md) +- [DROP](../../sql-reference/statements/drop.md) +- [EXISTS](../../sql-reference/statements/exists.md) +- [KILL](../../sql-reference/statements/kill.md) +- [OPTIMIZE](../../sql-reference/statements/optimize.md) +- [RENAME](../../sql-reference/statements/rename.md) +- [SET](../../sql-reference/statements/set.md) +- [SET ROLE](../../sql-reference/statements/set-role.md) +- [TRUNCATE](../../sql-reference/statements/truncate.md) +- [USE](../../sql-reference/statements/use.md) From 172bc46c630257532eb893b8d8fbc1d12316b854 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 16 Sep 2020 14:52:59 +0800 Subject: [PATCH 107/273] Fix format and dryrun event parse failure --- src/Core/MySQL/MySQLReplication.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index a46d787b225..c7593799caf 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -50,14 +50,13 @@ namespace MySQLReplication { payload.readStrict(reinterpret_cast(&binlog_version), 2); assert(binlog_version == EVENT_VERSION_V4); + server_version.resize(50); payload.readStrict(reinterpret_cast(server_version.data()), 50); payload.readStrict(reinterpret_cast(&create_timestamp), 4); payload.readStrict(reinterpret_cast(&event_header_length), 1); assert(event_header_length == EVENT_HEADER_LENGTH); - size_t len = header.event_size - (2 + 50 + 4 + 1 + EVENT_HEADER_LENGTH) - 1; - event_type_header_length.resize(len); - payload.readStrict(reinterpret_cast(event_type_header_length.data()), len); + readStringUntilEOF(event_type_header_length, payload); } void FormatDescriptionEvent::dump(std::ostream & out) const @@ -750,7 +749,7 @@ namespace MySQLReplication out << "GTID Next: " << gtid_next << std::endl; } - void DryRunEvent::parseImpl(ReadBuffer & payload) { payload.ignore(header.event_size - EVENT_HEADER_LENGTH); } + void DryRunEvent::parseImpl(ReadBuffer & payload) { payload.ignoreAll(); } void DryRunEvent::dump(std::ostream & out) const { From 9246e77b05654efbe9dd773580f187db56689784 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 16 Sep 2020 15:05:44 +0800 Subject: [PATCH 108/273] Remove CHECKSUM_CRC32_SIGNATURE_LENGTH in mysql event parser --- src/Core/MySQL/MySQLReplication.cpp | 4 +--- src/Core/MySQL/MySQLReplication.h | 1 - 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index c7593799caf..c874f0aad67 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -72,9 +72,7 @@ namespace MySQLReplication void RotateEvent::parseImpl(ReadBuffer & payload) { payload.readStrict(reinterpret_cast(&position), 8); - size_t len = header.event_size - EVENT_HEADER_LENGTH - 8 - CHECKSUM_CRC32_SIGNATURE_LENGTH; - next_binlog.resize(len); - payload.readStrict(reinterpret_cast(next_binlog.data()), len); + readStringUntilEOF(next_binlog, payload); } void RotateEvent::dump(std::ostream & out) const diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index ad5e53ed200..5f5ff23d0d9 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -19,7 +19,6 @@ namespace MySQLReplication { static const int EVENT_VERSION_V4 = 4; static const int EVENT_HEADER_LENGTH = 19; - static const int CHECKSUM_CRC32_SIGNATURE_LENGTH = 4; using Bitmap = boost::dynamic_bitset<>; From a8e671f0150947f96da602ca90b1f45972f397eb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 16 Sep 2020 11:58:27 +0300 Subject: [PATCH 109/273] Fix MutationsInterpreter. --- src/Interpreters/MutationsInterpreter.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 7f6f3441f5b..30da0d6e65f 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -724,9 +724,7 @@ void MutationsInterpreter::validate() QueryPlan plan; select_interpreter->buildQueryPlan(plan); - addStreamsForLaterStages(stages, plan); - - auto pipeline = plan.buildQueryPipeline(); + auto pipeline = addStreamsForLaterStages(stages, plan); } BlockInputStreamPtr MutationsInterpreter::execute() @@ -737,9 +735,7 @@ BlockInputStreamPtr MutationsInterpreter::execute() QueryPlan plan; select_interpreter->buildQueryPlan(plan); - addStreamsForLaterStages(stages, plan); - - auto pipeline = plan.buildQueryPipeline(); + auto pipeline = addStreamsForLaterStages(stages, plan); BlockInputStreamPtr result_stream = std::make_shared(std::move(*pipeline)); /// Sometimes we update just part of columns (for example UPDATE mutation) From bcea99f2e5104aa6437bf1e2456277c71a99b307 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 16 Sep 2020 11:59:58 +0300 Subject: [PATCH 110/273] Update ThreadPool.cpp --- src/Common/ThreadPool.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index cb8a7669eef..1255e3d11f8 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -279,7 +279,7 @@ void GlobalThreadPool::initialize(size_t max_threads) { if (the_instance) { - throw Exception(DB::ErrorCodes::LOGICAL_ERROR, + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "The global thread pool is initialized twice"); } From c5c5faf8ee46228288a1efc70baf794aac27b363 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 16 Sep 2020 12:06:42 +0300 Subject: [PATCH 111/273] 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 112/273] 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 38d53c38f6ad2b7298f4be7ab8c398abb6ee36d8 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 16 Sep 2020 14:36:55 +0800 Subject: [PATCH 113/273] Explicit define what first replica is. --- docs/en/operations/settings/settings.md | 2 ++ src/Client/ConnectionPoolWithFailover.cpp | 10 ++++++++-- src/Core/Settings.h | 1 + 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 76fcfa2a616..596095c3df9 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -940,6 +940,8 @@ This algorithm chooses the first replica in the set or a random replica if the f The `first_or_random` algorithm solves the problem of the `in_order` algorithm. With `in_order`, if one replica goes down, the next one gets a double load while the remaining replicas handle the usual amount of traffic. When using the `first_or_random` algorithm, the load is evenly distributed among replicas that are still available. +It's possible to explicitly define what the first replica is by using the setting `load_balancing_first_offset`. This gives more control to rebalance query workloads among replicas. + ### Round Robin {#load_balancing-round_robin} ``` sql diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index 6d6af794a07..68f4bcd1b76 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -56,6 +56,9 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts return tryGetEntry(pool, timeouts, fail_message, settings); }; + size_t offset = 0; + if (settings) + offset = settings->load_balancing_first_offset % nested_pools.size(); GetPriorityFunc get_priority; switch (settings ? LoadBalancing(settings->load_balancing) : default_load_balancing) { @@ -68,7 +71,7 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts case LoadBalancing::RANDOM: break; case LoadBalancing::FIRST_OR_RANDOM: - get_priority = [](size_t i) -> size_t { return i >= 1; }; + get_priority = [offset](size_t i) -> size_t { return i != offset; }; break; case LoadBalancing::ROUND_ROBIN: if (last_used >= nested_pools.size()) @@ -190,6 +193,9 @@ std::vector ConnectionPoolWithFailover::g else throw DB::Exception("Unknown pool allocation mode", DB::ErrorCodes::LOGICAL_ERROR); + size_t offset = 0; + if (settings) + offset = settings->load_balancing_first_offset % nested_pools.size(); GetPriorityFunc get_priority; switch (settings ? LoadBalancing(settings->load_balancing) : default_load_balancing) { @@ -202,7 +208,7 @@ std::vector ConnectionPoolWithFailover::g case LoadBalancing::RANDOM: break; case LoadBalancing::FIRST_OR_RANDOM: - get_priority = [](size_t i) -> size_t { return i >= 1; }; + get_priority = [offset](size_t i) -> size_t { return i != offset; }; break; case LoadBalancing::ROUND_ROBIN: if (last_used >= nested_pools.size()) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b39c223a5e9..6a6876d0f01 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -88,6 +88,7 @@ class IColumn; M(UInt64, replication_alter_columns_timeout, 60, "Wait for actions to change the table structure within the specified number of seconds. 0 - wait unlimited time.", 0) \ \ M(LoadBalancing, load_balancing, LoadBalancing::RANDOM, "Which replicas (among healthy replicas) to preferably send a query to (on the first attempt) for distributed processing.", 0) \ + M(UInt64, load_balancing_first_offset, 0, "Which replica to preferably send a query when FIRST_OR_RANDOM load balancing strategy is used.", 0) \ \ M(TotalsMode, totals_mode, TotalsMode::AFTER_HAVING_EXCLUSIVE, "How to calculate TOTALS when HAVING is present, as well as when max_rows_to_group_by and group_by_overflow_mode = ‘any’ are present.", IMPORTANT) \ M(Float, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.", 0) \ From da2a3fffe88665bb5c5e8e5e9546f1955cac4fd5 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 16 Sep 2020 13:00:15 +0300 Subject: [PATCH 114/273] fixup --- src/Dictionaries/CacheDictionary.cpp | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 29aee9bfc21..cb39dffeb6c 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -822,7 +822,24 @@ void CacheDictionary::waitForCurrentUpdateFinish(UpdateUnitPtr & update_unit_ptr if (update_unit_ptr->current_exception) - std::rethrow_exception(update_unit_ptr->current_exception); + { + // There might have been a single update unit for multiple callers in + // independent threads, and current_exception will be the same for them. + // Don't just rethrow it, because sharing the same exception object + // between multiple threads can lead to weird effects if they decide to + // modify it, for example, by adding some error context. + try + { + std::rethrow_exception(update_unit_ptr->current_exception); + } + catch (...) + { + throw DB::Exception(ErrorCodes::CACHE_DICTIONARY_UPDATE_FAIL, + "Dictionary update failed: {}", + getCurrentExceptionMessage(true /*with stack trace*/, + true /*check embedded stack trace*/)); + } + } } void CacheDictionary::tryPushToUpdateQueueOrThrow(UpdateUnitPtr & update_unit_ptr) const From 8be45dd3cc53017a033bc22fba373f47eaf69010 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 16 Sep 2020 13:25:01 +0300 Subject: [PATCH 115/273] Update QueryPipeline::addDelayingPipeline --- src/Processors/QueryPipeline.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 0337f19c2e3..c122dd240f7 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -264,16 +264,22 @@ QueryPipeline QueryPipeline::unitePipelines( void QueryPipeline::addDelayingPipeline(QueryPipeline pipeline) { - pipeline.resize(1); + checkInitializedAndNotCompleted(); + assertBlocksHaveEqualStructure(getHeader(), pipeline.getHeader(), "QueryPipeline"); + + IProcessor::PortNumbers delayed_streams(pipe.numOutputPorts()); + for (size_t i = 0; i < delayed_streams.size(); ++i) + delayed_streams[i] = i; auto * collected_processors = pipe.collected_processors; Pipes pipes; - pipes.emplace_back(QueryPipeline::getPipe(std::move(pipeline))); pipes.emplace_back(std::move(pipe)); + pipes.emplace_back(QueryPipeline::getPipe(std::move(pipeline))); pipe = Pipe::unitePipes(std::move(pipes), collected_processors); - pipe.addTransform(std::make_shared(getHeader(), 2)); + auto processor = std::make_shared(getHeader(), pipe.numOutputPorts(), delayed_streams); + addTransform(std::move(processor)); } void QueryPipeline::setProgressCallback(const ProgressCallback & callback) 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 116/273] 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 117/273] 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 8dc3c9b2394c8524a63b1819dda4573146f25dab Mon Sep 17 00:00:00 2001 From: Daria Mozhaeva Date: Wed, 16 Sep 2020 18:14:33 +0400 Subject: [PATCH 118/273] Edit and translate. --- .../aggregate-functions/reference/maxmap.md | 9 ++- .../aggregate-functions/reference/minmap.md | 2 +- .../functions/type-conversion-functions.md | 4 +- .../aggregate-functions/reference/maxmap.md | 28 +++++++ .../aggregate-functions/reference/minmap.md | 28 +++++++ .../functions/type-conversion-functions.md | 74 +++++++++++++++++++ 6 files changed, 139 insertions(+), 6 deletions(-) create mode 100644 docs/ru/sql-reference/aggregate-functions/reference/maxmap.md create mode 100644 docs/ru/sql-reference/aggregate-functions/reference/minmap.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/maxmap.md b/docs/en/sql-reference/aggregate-functions/reference/maxmap.md index 4dca13ed1b4..ffb36f06e61 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/maxmap.md +++ b/docs/en/sql-reference/aggregate-functions/reference/maxmap.md @@ -6,9 +6,12 @@ toc_priority: 143 Syntax: `maxMap(key, value)` or `maxMap(Tuple(key, value))` -Calculates the maximum from `value` array according to the keys specified in the ‘key’ array. -Passing tuple of keys and values arrays is synonymical to passing two arrays of keys and values. -The number of elements in ‘key’ and ‘value’ must be the same for each row that is totaled. +Calculates the maximum from `value` array according to the keys specified in the `key` array. + +Passing a tuple of keys and value ​​arrays is identical to passing two arrays of keys and values. + +The number of elements in `key` and `value` must be the same for each row that is totaled. + Returns a tuple of two arrays: keys in sorted order, and values calculated for the corresponding keys. Example: diff --git a/docs/en/sql-reference/aggregate-functions/reference/minmap.md b/docs/en/sql-reference/aggregate-functions/reference/minmap.md index 1b946dea209..9408d0ddfff 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/minmap.md +++ b/docs/en/sql-reference/aggregate-functions/reference/minmap.md @@ -8,7 +8,7 @@ Syntax: `minMap(key, value)` or `minMap(Tuple(key, value))` Calculates the minimum from `value` array according to the keys specified in the `key` array. -Passing tuple of keys and values arrays is a synonym to passing two arrays of keys and values. +Passing a tuple of keys and value ​​arrays is identical to passing two arrays of keys and values. The number of elements in `key` and `value` must be the same for each row that is totaled. diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 67361c350c7..515a4a650ce 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -523,7 +523,7 @@ Result: ## parseDateTimeBestEffortUS {#parsedatetimebesteffortUS} -This function is similar to [‘parseDateTimeBestEffort’](#parsedatetimebesteffort), the only difference is that this function prefers US style (`MM/DD/YYYY` etc) in case of ambiguouty. +This function is similar to [‘parseDateTimeBestEffort’](#parsedatetimebesteffort), the only difference is that this function prefers US date format (`MM/DD/YYYY` etc.) in case of ambiguity. **Syntax** @@ -541,7 +541,7 @@ parseDateTimeBestEffortUS(time_string [, time_zone]); - A string containing 9..10 digit [unix timestamp](https://en.wikipedia.org/wiki/Unix_time). - A string with a date and a time component: `YYYYMMDDhhmmss`, `MM/DD/YYYY hh:mm:ss`, `MM-DD-YY hh:mm`, `YYYY-MM-DD hh:mm:ss`, etc. - A string with a date, but no time component: `YYYY`, `YYYYMM`, `YYYY*MM`, `MM/DD/YYYY`, `MM-DD-YY` etc. -- A string with a day and time: `DD`, `DD hh`, `DD hh:mm`. In this case `YYYY-MM` are substituted as `2000-01`. +- A string with a day and time: `DD`, `DD hh`, `DD hh:mm`. In this case, `YYYY-MM` are substituted as `2000-01`. - A string that includes the date and time along with time zone offset information: `YYYY-MM-DD hh:mm:ss ±h:mm`, etc. For example, `2020-12-12 17:36:00 -5:00`. **Returned value** diff --git a/docs/ru/sql-reference/aggregate-functions/reference/maxmap.md b/docs/ru/sql-reference/aggregate-functions/reference/maxmap.md new file mode 100644 index 00000000000..efac771666b --- /dev/null +++ b/docs/ru/sql-reference/aggregate-functions/reference/maxmap.md @@ -0,0 +1,28 @@ +--- +toc_priority: 143 +--- + +# maxMap {#agg_functions-maxmap} + +Синтаксис: `maxMap(key, value)` or `maxMap(Tuple(key, value))` + +Вычисляет максимальное значение массива `value` в соответствии с ключами, указанными в массиве `key`. + +Передача кортежа ключей и массивов значений идентична передаче двух массивов ключей и значений. + +Количество элементов в параметрах `key` и `value` должно быть одинаковым для каждой суммируемой строки. + +Возвращает кортеж из двух массивов: ключи в отсортированном порядке и значения, рассчитанные для соответствующих ключей. + +Пример: + +``` sql +SELECT maxMap(a, b) +FROM values('a Array(Int32), b Array(Int64)', ([1, 2], [2, 2]), ([2, 3], [1, 1])) +``` + +``` text +┌─maxMap(a, b)──────┐ +│ ([1,2,3],[2,2,1]) │ +└───────────────────┘ +``` diff --git a/docs/ru/sql-reference/aggregate-functions/reference/minmap.md b/docs/ru/sql-reference/aggregate-functions/reference/minmap.md new file mode 100644 index 00000000000..e6def16e583 --- /dev/null +++ b/docs/ru/sql-reference/aggregate-functions/reference/minmap.md @@ -0,0 +1,28 @@ +--- +toc_priority: 142 +--- + +# minMap {#agg_functions-minmap} + +Синтаксис: `minMap(key, value)` or `minMap(Tuple(key, value))` + +Вычисляет минимальное значение массива `value` в соответствии с ключами, указанными в массиве `key`. + +Передача кортежа ключей и массивов значений идентична передаче двух массивов ключей и значений. + +Количество элементов в параметрах `key` и `value` должно быть одинаковым для каждой суммируемой строки. + +Возвращает кортеж из двух массивов: ключи в отсортированном порядке и значения, рассчитанные для соответствующих ключей. + +Пример: + +``` sql +SELECT minMap(a, b) +FROM values('a Array(Int32), b Array(Int64)', ([1, 2], [2, 2]), ([2, 3], [1, 1])) +``` + +``` text +┌─minMap(a, b)──────┐ +│ ([1,2,3],[2,1,1]) │ +└───────────────────┘ +``` diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index c7d74a9d881..7a57b94c4cd 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -513,6 +513,80 @@ SELECT parseDateTimeBestEffort('10 20:19') - [toDate](#todate) - [toDateTime](#todatetime) +## parseDateTimeBestEffortUS {#parsedatetimebesteffortUS} + +Эта функция похожа на [‘parseDateTimeBestEffort’](#parsedatetimebesteffort), но разница состоит в том, что в она использует американский формат даты (`MM/DD/YYYY` etc.) в случае многозначности. + +**Синтаксис** + +``` sql +parseDateTimeBestEffortUS(time_string [, time_zone]); +``` + +**Параметры** + +- `time_string` — строка, содержащая дату и время для преобразования. [String](../../sql-reference/data-types/string.md). +- `time_zone` — часовой пояс. Функция анализирует `time_string` в соответствии с часовым поясом. [String](../../sql-reference/data-types/string.md). + +**Поддерживаемые нестандартные форматы** + +- Строка, содержащая 9-10 цифр [unix timestamp](https://en.wikipedia.org/wiki/Unix_time). +- Строка, содержащая дату и время: `YYYYMMDDhhmmss`, `MM/DD/YYYY hh:mm:ss`, `MM-DD-YY hh:mm`, `YYYY-MM-DD hh:mm:ss`, etc. +- Строка с датой, но без времени: `YYYY`, `YYYYMM`, `YYYY*MM`, `MM/DD/YYYY`, `MM-DD-YY` etc. +- Строка, содержащая день и время: `DD`, `DD hh`, `DD hh:mm`. В этом случае `YYYY-MM` заменяется на `2000-01`. +- Строка, содержащая дату и время, а также информацию о часовом поясе: `YYYY-MM-DD hh:mm:ss ±h:mm` и т.д. Например, `2020-12-12 17:36:00 -5:00`. + +**Возвращаемое значение** + +- `time_string` преобразован в тип данных `DateTime`. + +**Примеры** + +Запрос: + +``` sql +SELECT parseDateTimeBestEffortUS('09/12/2020 12:12:57') +AS parseDateTimeBestEffortUS; +``` + +Ответ: + +``` text +┌─parseDateTimeBestEffortUS─┐ +│ 2020-09-12 12:12:57 │ +└─────────────────────────——┘ +``` + +Запрос: + +``` sql +SELECT parseDateTimeBestEffortUS('09-12-2020 12:12:57') +AS parseDateTimeBestEffortUS; +``` + +Ответ: + +``` text +┌─parseDateTimeBestEffortUS─┐ +│ 2020-09-12 12:12:57 │ +└─────────────────────────——┘ +``` + +Запрос: + +``` sql +SELECT parseDateTimeBestEffortUS('09.12.2020 12:12:57') +AS parseDateTimeBestEffortUS; +``` + +Ответ: + +``` text +┌─parseDateTimeBestEffortUS─┐ +│ 2020-09-12 12:12:57 │ +└─────────────────────────——┘ +``` + ## toUnixTimestamp64Milli ## toUnixTimestamp64Micro ## toUnixTimestamp64Nano From 8de76709fb5d5eb50f3668c69bea3fd440f63e40 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 16 Sep 2020 17:41:01 +0300 Subject: [PATCH 119/273] Fix tests. --- src/Processors/QueryPlan/CreatingSetsStep.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index f63b657c029..bf2a0aa1929 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -121,6 +121,9 @@ void addCreatingSetsStep( for (auto & [description, set] : subqueries_for_sets) { + if (!set.source) + continue; + auto plan = std::move(set.source); std::string type = (set.join != nullptr) ? "JOIN" : "subquery"; @@ -139,6 +142,12 @@ void addCreatingSetsStep( plans.emplace_back(std::move(plan)); } + if (plans.size() == 1) + { + query_plan = std::move(*plans.front()); + return; + } + auto creating_sets = std::make_unique(std::move(input_streams)); creating_sets->setStepDescription("Create sets before main query execution"); query_plan.unitePlans(std::move(creating_sets), std::move(plans)); 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 120/273] 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 8a26251f94f7bbeaa21b41e6d930d8458b530d25 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 16 Sep 2020 19:11:16 +0300 Subject: [PATCH 121/273] Fix tests. --- src/Processors/QueryPipeline.cpp | 20 +++++++++++++++++++ src/Processors/QueryPipeline.h | 2 ++ src/Processors/QueryPlan/CreatingSetsStep.cpp | 9 +-------- 3 files changed, 23 insertions(+), 8 deletions(-) diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index c122dd240f7..7f7e2391bde 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -262,6 +262,26 @@ QueryPipeline QueryPipeline::unitePipelines( return pipeline; } + +void QueryPipeline::addCreatingSetsTransform() +{ + pipeline.resize(1); + + auto transform = std::make_shared( + pipeline.getHeader(), + getOutputStream().header, + std::move(subquery_for_set), + network_transfer_limits, + context)); + + InputPort * totals_port = nullptr; + + if (pipe.getTotalsPort()) + totals_port = transform->addTotalsPort(); + + pipe.addTransform(std::move(transform), totals_port, nullptr); +} + void QueryPipeline::addDelayingPipeline(QueryPipeline pipeline) { checkInitializedAndNotCompleted(); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 4be7bbfd573..e2083ddae66 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -89,6 +89,8 @@ public: /// Pipeline must have same header. void addDelayingPipeline(QueryPipeline pipeline); + void addCreatingSetsTransform(); + PipelineExecutorPtr execute(); size_t getNumStreams() const { return pipe.numOutputPorts(); } diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index bf2a0aa1929..6a7cfc5ced8 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -39,14 +39,7 @@ CreatingSetStep::CreatingSetStep( void CreatingSetStep::transformPipeline(QueryPipeline & pipeline) { - pipeline.resize(1); - pipeline.addTransform( - std::make_shared( - pipeline.getHeader(), - getOutputStream().header, - std::move(subquery_for_set), - network_transfer_limits, - context)); + pipeline.addCreatingSetsTransform(); } void CreatingSetStep::describeActions(FormatSettings & settings) const From 2f7ab2bca4715b0c2f13547b5ee7a0e26b8f48df Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 16 Sep 2020 19:30:48 +0300 Subject: [PATCH 122/273] Fix tests. --- src/Processors/QueryPipeline.cpp | 6 +++--- src/Processors/QueryPipeline.h | 4 +++- src/Processors/QueryPlan/CreatingSetsStep.cpp | 2 +- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 7f7e2391bde..bef0c0a3047 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -263,15 +263,15 @@ QueryPipeline QueryPipeline::unitePipelines( } -void QueryPipeline::addCreatingSetsTransform() +void QueryPipeline::addCreatingSetsTransform(SubqueryForSet subquery_for_set, const SizeLimits & limits, const Context & context) { - pipeline.resize(1); + resize(1); auto transform = std::make_shared( pipeline.getHeader(), getOutputStream().header, std::move(subquery_for_set), - network_transfer_limits, + limits, context)); InputPort * totals_port = nullptr; diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index e2083ddae66..66947bc9fa5 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -26,6 +26,8 @@ class QueryPlan; struct SubqueryForSet; using SubqueriesForSets = std::unordered_map; +struct SizeLimits; + class QueryPipeline { public: @@ -89,7 +91,7 @@ public: /// Pipeline must have same header. void addDelayingPipeline(QueryPipeline pipeline); - void addCreatingSetsTransform(); + void addCreatingSetsTransform(SubqueryForSet subquery_for_set, const SizeLimits & limits, const Context & context); PipelineExecutorPtr execute(); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 6a7cfc5ced8..4de11d0791d 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -39,7 +39,7 @@ CreatingSetStep::CreatingSetStep( void CreatingSetStep::transformPipeline(QueryPipeline & pipeline) { - pipeline.addCreatingSetsTransform(); + pipeline.addCreatingSetsTransform(std::move(subquery_for_set), network_transfer_limits, context); } void CreatingSetStep::describeActions(FormatSettings & settings) const From 73ad5057357e4e2e41c52fad9fda3308b05aed43 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 16 Sep 2020 19:38:33 +0300 Subject: [PATCH 123/273] Fix tests. --- src/Processors/QueryPipeline.cpp | 4 ++-- src/Processors/QueryPipeline.h | 2 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index bef0c0a3047..598bb0c8053 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -263,13 +263,13 @@ QueryPipeline QueryPipeline::unitePipelines( } -void QueryPipeline::addCreatingSetsTransform(SubqueryForSet subquery_for_set, const SizeLimits & limits, const Context & context) +void QueryPipeline::addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, const Context & context) { resize(1); auto transform = std::make_shared( pipeline.getHeader(), - getOutputStream().header, + res_header, std::move(subquery_for_set), limits, context)); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 66947bc9fa5..2a318a53b76 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -91,7 +91,7 @@ public: /// Pipeline must have same header. void addDelayingPipeline(QueryPipeline pipeline); - void addCreatingSetsTransform(SubqueryForSet subquery_for_set, const SizeLimits & limits, const Context & context); + void addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, const Context & context); PipelineExecutorPtr execute(); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 4de11d0791d..27ba3153151 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -39,7 +39,7 @@ CreatingSetStep::CreatingSetStep( void CreatingSetStep::transformPipeline(QueryPipeline & pipeline) { - pipeline.addCreatingSetsTransform(std::move(subquery_for_set), network_transfer_limits, context); + pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, context); } void CreatingSetStep::describeActions(FormatSettings & settings) const From fe1056579f0563c7b427b2baf2bae59fd568430f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 16 Sep 2020 19:45:56 +0300 Subject: [PATCH 124/273] Fix tests. --- src/Processors/QueryPipeline.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 598bb0c8053..c8218023a57 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -268,11 +268,11 @@ void QueryPipeline::addCreatingSetsTransform(const Block & res_header, SubqueryF resize(1); auto transform = std::make_shared( - pipeline.getHeader(), + getHeader(), res_header, std::move(subquery_for_set), limits, - context)); + context); InputPort * totals_port = nullptr; From e1736584225ee8c36c7eb4ac6eba024c470850ce Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 16 Sep 2020 19:53:58 +0300 Subject: [PATCH 125/273] Fix integration test --- .../test_replicated_merge_tree_config/test.py | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_replicated_merge_tree_config/test.py b/tests/integration/test_replicated_merge_tree_config/test.py index bcf6517782d..e3bc88b155f 100644 --- a/tests/integration/test_replicated_merge_tree_config/test.py +++ b/tests/integration/test_replicated_merge_tree_config/test.py @@ -1,14 +1,14 @@ import pytest from helpers.cluster import ClickHouseCluster +import logging + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node", main_configs=["configs/config.xml"], with_zookeeper=True) @pytest.fixture(scope="module") -def cluster(): +def start_cluster(): try: - cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "node", config_dir="configs", with_zookeeper=True, - ) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -19,19 +19,18 @@ def cluster(): @pytest.fixture(autouse=True) -def drop_table(cluster): +def drop_table(start_cluster): yield for node in cluster.instances.values(): node.query("DROP TABLE IF EXISTS test1") node.query("DROP TABLE IF EXISTS test2") -def test_replicated_merge_tree_settings(cluster): - node = cluster.instances["node"] +def test_replicated_merge_tree_settings(start_cluster): node.query("CREATE TABLE test1 (id Int64) ENGINE MergeTree ORDER BY id") node.query( "CREATE TABLE test2 (id Int64) ENGINE ReplicatedMergeTree('/clickhouse/test', 'test') ORDER BY id" ) - assert node.query("SHOW CREATE test1").endswith("100") - assert node.query("SHOW CREATE test2").endswith("200") + assert node.query("SHOW CREATE test1").strip().endswith("100") + assert node.query("SHOW CREATE test2").strip().endswith("200") From d0debd51c0263fd00cc49d391cf499c733ad2c7a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 16 Sep 2020 20:00:44 +0300 Subject: [PATCH 126/273] Add sorting to fix test. --- tests/queries/0_stateless/00853_join_with_nulls_crash.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00853_join_with_nulls_crash.sql b/tests/queries/0_stateless/00853_join_with_nulls_crash.sql index eb64ed29ffe..464ddbb1990 100644 --- a/tests/queries/0_stateless/00853_join_with_nulls_crash.sql +++ b/tests/queries/0_stateless/00853_join_with_nulls_crash.sql @@ -21,14 +21,14 @@ SELECT s1.other, s2.other, count_a, count_b, toTypeName(s1.other), toTypeName(s2 ALL FULL JOIN ( SELECT other, count() AS count_b FROM table_b GROUP BY other ) s2 ON s1.other = s2.other -ORDER BY s2.other DESC; +ORDER BY s2.other DESC, count_a; SELECT s1.other, s2.other, count_a, count_b, toTypeName(s1.other), toTypeName(s2.other) FROM ( SELECT other, count() AS count_a FROM table_a GROUP BY other ) s1 ALL FULL JOIN ( SELECT other, count() AS count_b FROM table_b GROUP BY other ) s2 USING other -ORDER BY s2.other DESC; +ORDER BY s2.other DESC, count_a; SELECT s1.something, s2.something, count_a, count_b, toTypeName(s1.something), toTypeName(s2.something) FROM ( SELECT something, count() AS count_a FROM table_a GROUP BY something ) s1 From 7d046b24e653a5d19737b422a25456ae8b50918d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 16 Sep 2020 23:56:16 +0300 Subject: [PATCH 127/273] 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 bf9feb68659fc3c04022b35f751f4c80249694ed Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Wed, 16 Sep 2020 22:07:58 -0400 Subject: [PATCH 128/273] Removing usage of time.sleep in tests/integration/test_distributed_over_live_view/test.py --- .../test_distributed_over_live_view/test.py | 55 +++++++++++++------ 1 file changed, 38 insertions(+), 17 deletions(-) diff --git a/tests/integration/test_distributed_over_live_view/test.py b/tests/integration/test_distributed_over_live_view/test.py index 46c0bada535..ed6b8be23df 100644 --- a/tests/integration/test_distributed_over_live_view/test.py +++ b/tests/integration/test_distributed_over_live_view/test.py @@ -60,6 +60,13 @@ def started_cluster(): finally: cluster.shutdown() +def poll_query(node, query, expected, timeout): + """Repeatedly execute query until either expected result is returned or timeout occurs. + """ + start_time = time.time() + while node.query(query) != expected and time.time() - start_time < timeout: + pass + assert node.query(query) == expected @pytest.mark.parametrize("node", NODES.values()[:1]) @pytest.mark.parametrize("source", ["lv_over_distributed_table"]) @@ -69,6 +76,8 @@ class TestLiveViewOverDistributedSuite: node0, node1 = NODES.values() select_query = "SELECT * FROM distributed_over_lv ORDER BY node, key FORMAT CSV" + select_query_dist_table = "SELECT * FROM distributed_table ORDER BY node, key FORMAT CSV" + select_count_query = "SELECT count() FROM distributed_over_lv" with client(name="client1> ", log=log, command=" ".join(node0.client.command)) as client1, \ client(name="client2> ", log=log, command=" ".join(node1.client.command)) as client2: @@ -91,7 +100,17 @@ class TestLiveViewOverDistributedSuite: client1.expect(prompt) client2.send("INSERT INTO distributed_table VALUES ('node1', 3, 3)") client2.expect(prompt) - time.sleep(2) + + poll_query(node0, select_count_query, "7\n", timeout=60) + print("\n--DEBUG1--") + print(select_query) + print(node0.query(select_query)) + print("---------") + print("\n--DEBUG2--") + print(select_query_dist_table) + print(node0.query(select_query_dist_table)) + print("---------") + client1.send(select_query) client1.expect('"node1",0,0') client1.expect('"node1",1,1') @@ -107,6 +126,7 @@ class TestLiveViewOverDistributedSuite: node0, node1 = NODES.values() select_query = "SELECT * FROM distributed_over_lv ORDER BY key, node FORMAT CSV" + select_count_query = "SELECT count() FROM distributed_over_lv" with client(name="client1> ", log=log, command=" ".join(node0.client.command)) as client1, \ client(name="client2> ", log=log, command=" ".join(node1.client.command)) as client2: @@ -129,7 +149,9 @@ class TestLiveViewOverDistributedSuite: client1.expect(prompt) client2.send("INSERT INTO distributed_table VALUES ('node1', 3, 3)") client2.expect(prompt) - time.sleep(2) + + poll_query(node0, select_count_query, "7\n", timeout=60) + client1.send(select_query) client1.expect('"node1",0,0') client1.expect('"node2",0,10') @@ -163,7 +185,9 @@ class TestLiveViewOverDistributedSuite: client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)") client2.expect(prompt) - time.sleep(2) + + poll_query(node0, select_query, '"node1",3\n"node2",21\n', timeout=60) + client1.send(select_query) client1.expect('"node1",3') client1.expect('"node2",21') @@ -173,7 +197,9 @@ class TestLiveViewOverDistributedSuite: client1.expect(prompt) client2.send("INSERT INTO distributed_table VALUES ('node1', 3, 3)") client2.expect(prompt) - time.sleep(2) + + poll_query(node0, select_query, '"node1",12\n"node2",21\n', timeout=60) + client1.send(select_query) client1.expect('"node1",12') client1.expect('"node2",21') @@ -202,7 +228,9 @@ class TestLiveViewOverDistributedSuite: client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)") client2.expect(prompt) - time.sleep(2) + + poll_query(node0, "SELECT count() FROM (%s)" % select_query.rsplit("FORMAT")[0], "3\n", timeout=60) + client1.send(select_query) client1.expect('0,10') client1.expect('1,12') @@ -211,7 +239,9 @@ class TestLiveViewOverDistributedSuite: client2.send("INSERT INTO distributed_table VALUES ('node1', 1, 3), ('node1', 3, 3)") client2.expect(prompt) - time.sleep(2) + + poll_query(node0, "SELECT count() FROM (%s)" % select_query.rsplit("FORMAT")[0], "4\n", timeout=60) + client1.send(select_query) client1.expect('0,10') client1.expect('1,15') @@ -240,18 +270,9 @@ class TestLiveViewOverDistributedSuite: client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)") client2.expect(prompt) - time.sleep(2) - - client1.send("SELECT sum(value) FROM distributed_over_lv") - client1.expect(r"24" + end_of_block) - client1.expect(prompt) + poll_query(node0, "SELECT sum(value) FROM distributed_over_lv", "24\n", timeout=60) client2.send("INSERT INTO distributed_table VALUES ('node1', 3, 3), ('node1', 4, 4)") client2.expect(prompt) - time.sleep(2) - - client1.send("SELECT sum(value) FROM distributed_over_lv") - client1.expect(r"31" + end_of_block) - client1.expect(prompt) - + poll_query(node0, "SELECT sum(value) FROM distributed_over_lv", "31\n", timeout=60) From 13088d9befdd0e2d91764a6ba89c80bb88227138 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 17 Sep 2020 08:04:50 +0300 Subject: [PATCH 129/273] 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 73847331444e4cade1709345f5ad2ec6f863dddf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 17 Sep 2020 09:01:35 +0300 Subject: [PATCH 130/273] Fix style. --- src/Processors/QueryPlan/CreatingSetsStep.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 27ba3153151..780e8493b40 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -6,6 +6,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + static ITransformingStep::Traits getTraits() { return ITransformingStep::Traits From 0c81a8777e17705032d69a1744fb8f6d9e5201b2 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 17 Sep 2020 14:21:38 +0800 Subject: [PATCH 131/273] Fix review comment --- src/Core/MySQL/MySQLReplication.cpp | 51 ++++++++++++++--------------- src/Core/MySQL/MySQLReplication.h | 34 ++++++++++++------- 2 files changed, 46 insertions(+), 39 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index c874f0aad67..824440b3dd0 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -15,6 +15,7 @@ namespace ErrorCodes { extern const int UNKNOWN_EXCEPTION; extern const int LOGICAL_ERROR; + extern const int ATTEMPT_TO_READ_AFTER_EOF; } namespace MySQLReplication @@ -800,6 +801,9 @@ namespace MySQLReplication void MySQLFlavor::readPayloadImpl(ReadBuffer & payload) { + if (payload.eof()) + throw Exception("Attempt to read after EOF.", ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF); + UInt16 header = static_cast(*payload.position()); switch (header) { @@ -810,33 +814,33 @@ namespace MySQLReplication err.readPayloadWithUnpacked(payload); throw ReplicationError(err.error_message, ErrorCodes::UNKNOWN_EXCEPTION); } - // skip the header flag. + // skip the generic response packets header flag. payload.ignore(1); MySQLBinlogEventReadBuffer event_payload(payload); - EventType event_type = static_cast(*(event_payload.position() + 4)); - switch (event_type) + + EventHeader event_header; + event_header.parse(event_payload); + + switch (event_header.type) { case FORMAT_DESCRIPTION_EVENT: { - event = std::make_shared(); - event->parseHeader(event_payload); + event = std::make_shared(std::move(event_header)); event->parseEvent(event_payload); position.update(event); break; } case ROTATE_EVENT: { - event = std::make_shared(); - event->parseHeader(event_payload); + event = std::make_shared(std::move(event_header)); event->parseEvent(event_payload); position.update(event); break; } case QUERY_EVENT: { - event = std::make_shared(); - event->parseHeader(event_payload); + event = std::make_shared(std::move(event_header)); event->parseEvent(event_payload); auto query = std::static_pointer_cast(event); @@ -845,7 +849,7 @@ namespace MySQLReplication case QUERY_EVENT_MULTI_TXN_FLAG: case QUERY_EVENT_XA: { - event = std::make_shared(); + event = std::make_shared(std::move(query->header)); break; } default: @@ -855,16 +859,14 @@ namespace MySQLReplication } case XID_EVENT: { - event = std::make_shared(); - event->parseHeader(event_payload); + event = std::make_shared(std::move(event_header)); event->parseEvent(event_payload); position.update(event); break; } case TABLE_MAP_EVENT: { - event = std::make_shared(); - event->parseHeader(event_payload); + event = std::make_shared(std::move(event_header)); event->parseEvent(event_payload); table_map = std::static_pointer_cast(event); break; @@ -873,11 +875,10 @@ namespace MySQLReplication case WRITE_ROWS_EVENT_V2: { if (do_replicate()) - event = std::make_shared(table_map); + event = std::make_shared(table_map, std::move(event_header)); else - event = std::make_shared(); + event = std::make_shared(std::move(event_header)); - event->parseHeader(event_payload); event->parseEvent(event_payload); break; } @@ -885,11 +886,10 @@ namespace MySQLReplication case DELETE_ROWS_EVENT_V2: { if (do_replicate()) - event = std::make_shared(table_map); + event = std::make_shared(table_map, std::move(event_header)); else - event = std::make_shared(); + event = std::make_shared(std::move(event_header)); - event->parseHeader(event_payload); event->parseEvent(event_payload); break; } @@ -897,26 +897,23 @@ namespace MySQLReplication case UPDATE_ROWS_EVENT_V2: { if (do_replicate()) - event = std::make_shared(table_map); + event = std::make_shared(table_map, std::move(event_header)); else - event = std::make_shared(); + event = std::make_shared(std::move(event_header)); - event->parseHeader(event_payload); event->parseEvent(event_payload); break; } case GTID_EVENT: { - event = std::make_shared(); - event->parseHeader(event_payload); + event = std::make_shared(std::move(event_header)); event->parseEvent(event_payload); position.update(event); break; } default: { - event = std::make_shared(); - event->parseHeader(event_payload); + event = std::make_shared(std::move(event_header)); event->parseEvent(event_payload); break; } diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index 5f5ff23d0d9..fbcd67105ed 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -300,9 +300,10 @@ namespace MySQLReplication public: EventHeader header; + EventBase(EventHeader && header_) : header(std::move(header_)) {} + virtual ~EventBase() = default; virtual void dump(std::ostream & out) const = 0; - virtual void parseHeader(ReadBuffer & payload) { header.parse(payload); } virtual void parseEvent(ReadBuffer & payload) { parseImpl(payload); } virtual MySQLEventType type() const { return MYSQL_UNHANDLED_EVENT; } @@ -313,7 +314,10 @@ namespace MySQLReplication class FormatDescriptionEvent : public EventBase { public: - FormatDescriptionEvent() : binlog_version(0), create_timestamp(0), event_header_length(0) { } + FormatDescriptionEvent(EventHeader && header_) + : EventBase(std::move(header_)), binlog_version(0), create_timestamp(0), event_header_length(0) + { + } protected: UInt16 binlog_version; @@ -335,7 +339,7 @@ namespace MySQLReplication UInt64 position; String next_binlog; - RotateEvent() : position(0) { } + RotateEvent(EventHeader && header_) : EventBase(std::move(header_)), position(0) {} void dump(std::ostream & out) const override; protected: @@ -362,7 +366,11 @@ namespace MySQLReplication String query; QueryType typ = QUERY_EVENT_DDL; - QueryEvent() : thread_id(0), exec_time(0), schema_len(0), error_code(0), status_len(0) { } + QueryEvent(EventHeader && header_) + : EventBase(std::move(header_)), thread_id(0), exec_time(0), schema_len(0), error_code(0), status_len(0) + { + } + void dump(std::ostream & out) const override; MySQLEventType type() const override { return MYSQL_QUERY_EVENT; } @@ -373,7 +381,7 @@ namespace MySQLReplication class XIDEvent : public EventBase { public: - XIDEvent() : xid(0) { } + XIDEvent(EventHeader && header_) : EventBase(std::move(header_)), xid(0) {} protected: UInt64 xid; @@ -396,7 +404,7 @@ namespace MySQLReplication std::vector column_meta; Bitmap null_bitmap; - TableMapEvent() : table_id(0), flags(0), schema_len(0), table_len(0), column_count(0) { } + TableMapEvent(EventHeader && header_) : EventBase(std::move(header_)), table_id(0), flags(0), schema_len(0), table_len(0), column_count(0) {} void dump(std::ostream & out) const override; protected: @@ -412,8 +420,8 @@ namespace MySQLReplication String table; std::vector rows; - RowsEvent(std::shared_ptr table_map_) - : number_columns(0), table_id(0), flags(0), extra_data_len(0), table_map(table_map_) + RowsEvent(std::shared_ptr table_map_, EventHeader && header_) + : EventBase(std::move(header_)), number_columns(0), table_id(0), flags(0), extra_data_len(0), table_map(table_map_) { schema = table_map->schema; table = table_map->table; @@ -438,21 +446,21 @@ namespace MySQLReplication class WriteRowsEvent : public RowsEvent { public: - WriteRowsEvent(std::shared_ptr table_map_) : RowsEvent(table_map_) { } + WriteRowsEvent(std::shared_ptr table_map_, EventHeader && header_) : RowsEvent(table_map_, std::move(header_)) {} MySQLEventType type() const override { return MYSQL_WRITE_ROWS_EVENT; } }; class DeleteRowsEvent : public RowsEvent { public: - DeleteRowsEvent(std::shared_ptr table_map_) : RowsEvent(table_map_) { } + DeleteRowsEvent(std::shared_ptr table_map_, EventHeader && header_) : RowsEvent(table_map_, std::move(header_)) {} MySQLEventType type() const override { return MYSQL_DELETE_ROWS_EVENT; } }; class UpdateRowsEvent : public RowsEvent { public: - UpdateRowsEvent(std::shared_ptr table_map_) : RowsEvent(table_map_) { } + UpdateRowsEvent(std::shared_ptr table_map_, EventHeader && header_) : RowsEvent(table_map_, std::move(header_)) {} MySQLEventType type() const override { return MYSQL_UPDATE_ROWS_EVENT; } }; @@ -462,7 +470,7 @@ namespace MySQLReplication UInt8 commit_flag; GTID gtid; - GTIDEvent() : commit_flag(0) { } + GTIDEvent(EventHeader && header_) : EventBase(std::move(header_)), commit_flag(0) {} void dump(std::ostream & out) const override; protected: @@ -471,6 +479,8 @@ namespace MySQLReplication class DryRunEvent : public EventBase { + public: + DryRunEvent(EventHeader && header_) : EventBase(std::move(header_)) {} void dump(std::ostream & out) const override; protected: From c5b56c24f83bde0a70e4304bf13d4e723218db67 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 17 Sep 2020 14:44:05 +0800 Subject: [PATCH 132/273] ISSUES-14894 fix MaterializeMySQL temp metadata file exists --- src/Databases/MySQL/MaterializeMetadata.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 74fd59dc98e..3c5bfdec594 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -145,7 +145,7 @@ void MaterializeMetadata::transaction(const MySQLReplication::Position & positio String persistent_tmp_path = persistent_path + ".tmp"; { - WriteBufferFromFile out(persistent_tmp_path, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_TRUNC | O_CREAT | O_EXCL); + WriteBufferFromFile out(persistent_tmp_path, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_TRUNC | O_CREAT); /// TSV format metadata file. writeString("Version:\t" + toString(meta_version), out); From 339521fadbfc3a82a8a0b3f2fae5ac1ec48ca7b1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 17 Sep 2020 11:48:27 +0300 Subject: [PATCH 133/273] Update test.py --- tests/integration/test_replicated_merge_tree_config/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_replicated_merge_tree_config/test.py b/tests/integration/test_replicated_merge_tree_config/test.py index e3bc88b155f..2a7725960bf 100644 --- a/tests/integration/test_replicated_merge_tree_config/test.py +++ b/tests/integration/test_replicated_merge_tree_config/test.py @@ -32,5 +32,5 @@ def test_replicated_merge_tree_settings(start_cluster): "CREATE TABLE test2 (id Int64) ENGINE ReplicatedMergeTree('/clickhouse/test', 'test') ORDER BY id" ) - assert node.query("SHOW CREATE test1").strip().endswith("100") - assert node.query("SHOW CREATE test2").strip().endswith("200") + assert "index_granularity = 100" in node.query("SHOW CREATE test1") + assert "index_granularity = 200" in node.query("SHOW CREATE test2") From b41421cb1cdc6c3f86ef46f27521612621eef3a3 Mon Sep 17 00:00:00 2001 From: roman Date: Fri, 28 Aug 2020 15:50:25 +0100 Subject: [PATCH 134/273] [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 135/273] 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 72c68a8746bc22ec8e258fcd2e177833c8b867d7 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 17 Sep 2020 13:05:03 +0300 Subject: [PATCH 136/273] boop the CI From ada1c7c4a03600de35f6031840dae25c9622025c Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 17 Sep 2020 13:15:46 +0300 Subject: [PATCH 137/273] 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 138/273] 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 139/273] 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 140/273] 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 141/273] 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 8bce20076c0e2b38b5957cd9bc3021516030132a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 15:39:37 +0300 Subject: [PATCH 142/273] Only mlock code segment --- programs/server/Server.cpp | 48 ++++++++++++++++++---------------- src/Common/remapExecutable.cpp | 47 ++++++++++++++++----------------- src/Common/remapExecutable.h | 3 +++ 3 files changed, 52 insertions(+), 46 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 56778b8dd69..2baca4c7379 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -90,6 +90,23 @@ namespace CurrentMetrics extern const Metric MemoryTracking; } + +int mainEntryClickHouseServer(int argc, char ** argv) +{ + DB::Server app; + try + { + return app.run(argc, argv); + } + catch (...) + { + std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; + auto code = DB::getCurrentExceptionCode(); + return code ? code : 1; + } +} + + namespace { @@ -317,11 +334,16 @@ int Server::main(const std::vector & /*args*/) { if (hasLinuxCapability(CAP_IPC_LOCK)) { - LOG_TRACE(log, "Will mlockall to prevent executable memory from being paged out. It may take a few seconds."); - if (0 != mlockall(MCL_CURRENT)) - LOG_WARNING(log, "Failed mlockall: {}", errnoToString(ErrorCodes::SYSTEM_ERROR)); + /// Get the memory area with (current) code segment. + /// It's better to lock only the code segment instead of calling "mlockall", + /// because otherwise debug info will be also locked in memory, and it can be huge. + auto [addr, len] = getMappedArea(reinterpret_cast(mainEntryClickHouseServer)); + + LOG_TRACE(log, "Will do mlock to prevent executable memory from being paged out. It may take a few seconds."); + if (0 != mlock(addr, len)) + LOG_WARNING(log, "Failed mlock: {}", errnoToString(ErrorCodes::SYSTEM_ERROR)); else - LOG_TRACE(log, "The memory map of clickhouse executable has been mlock'ed"); + LOG_TRACE(log, "The memory map of clickhouse executable has been mlock'ed, total {}", ReadableSize(len)); } else { @@ -1132,21 +1154,3 @@ int Server::main(const std::vector & /*args*/) return Application::EXIT_OK; } } - -#pragma GCC diagnostic ignored "-Wunused-function" -#pragma GCC diagnostic ignored "-Wmissing-declarations" - -int mainEntryClickHouseServer(int argc, char ** argv) -{ - DB::Server app; - try - { - return app.run(argc, argv); - } - catch (...) - { - std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; - auto code = DB::getCurrentExceptionCode(); - return code ? code : 1; - } -} diff --git a/src/Common/remapExecutable.cpp b/src/Common/remapExecutable.cpp index 13bce459022..35d0338f9ce 100644 --- a/src/Common/remapExecutable.cpp +++ b/src/Common/remapExecutable.cpp @@ -48,30 +48,6 @@ uintptr_t readAddressHex(DB::ReadBuffer & in) } -/** Find the address and size of the mapped memory region pointed by ptr. - */ -std::pair getMappedArea(void * ptr) -{ - using namespace DB; - - uintptr_t uintptr = reinterpret_cast(ptr); - ReadBufferFromFile in("/proc/self/maps"); - - while (!in.eof()) - { - uintptr_t begin = readAddressHex(in); - assertChar('-', in); - uintptr_t end = readAddressHex(in); - skipToNextLineOrEOF(in); - - if (begin <= uintptr && uintptr < end) - return {reinterpret_cast(begin), end - begin}; - } - - throw Exception("Cannot find mapped area for pointer", ErrorCodes::LOGICAL_ERROR); -} - - __attribute__((__noinline__)) int64_t our_syscall(...) { __asm__ __volatile__ (R"( @@ -181,6 +157,28 @@ __attribute__((__noinline__)) void remapToHugeStep1(void * begin, size_t size) } +std::pair getMappedArea(void * ptr) +{ + using namespace DB; + + uintptr_t uintptr = reinterpret_cast(ptr); + ReadBufferFromFile in("/proc/self/maps"); + + while (!in.eof()) + { + uintptr_t begin = readAddressHex(in); + assertChar('-', in); + uintptr_t end = readAddressHex(in); + skipToNextLineOrEOF(in); + + if (begin <= uintptr && uintptr < end) + return {reinterpret_cast(begin), end - begin}; + } + + throw Exception("Cannot find mapped area for pointer", ErrorCodes::LOGICAL_ERROR); +} + + void remapExecutable() { auto [begin, size] = getMappedArea(reinterpret_cast(remapExecutable)); @@ -195,6 +193,7 @@ namespace DB { void remapExecutable() {} +std::pair getMappedArea(void * ptr) { return {}; } } diff --git a/src/Common/remapExecutable.h b/src/Common/remapExecutable.h index 7acb61f13bd..af0a96d5248 100644 --- a/src/Common/remapExecutable.h +++ b/src/Common/remapExecutable.h @@ -4,4 +4,7 @@ namespace DB /// This function tries to reallocate the code of the running program in a more efficient way. void remapExecutable(); +/// Find the address and size of the mapped memory region pointed by ptr. +std::pair getMappedArea(void * ptr); + } From 068e8576b7a930c4d7fdd9420a17e69c8b93eaaf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 15:53:52 +0300 Subject: [PATCH 143/273] Corrections --- programs/server/Server.cpp | 1 + src/Common/getMappedArea.cpp | 80 ++++++++++++++++++++++++++++++++++ src/Common/getMappedArea.h | 10 +++++ src/Common/remapExecutable.cpp | 51 ++-------------------- src/Common/remapExecutable.h | 3 -- 5 files changed, 94 insertions(+), 51 deletions(-) create mode 100644 src/Common/getMappedArea.cpp create mode 100644 src/Common/getMappedArea.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 2baca4c7379..0bb64a29a09 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Common/getMappedArea.cpp b/src/Common/getMappedArea.cpp new file mode 100644 index 00000000000..6817bea8cb1 --- /dev/null +++ b/src/Common/getMappedArea.cpp @@ -0,0 +1,80 @@ +#if defined(__linux__) + +#include +#include +#include +#include +#include + +#include "getMappedArea.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; +} + + +namespace +{ + +uintptr_t readAddressHex(DB::ReadBuffer & in) +{ + uintptr_t res = 0; + while (!in.eof()) + { + if (isHexDigit(*in.position())) + { + res *= 16; + res += unhex(*in.position()); + ++in.position(); + } + else + break; + } + return res; +} + +} + +std::pair getMappedArea(void * ptr) +{ + using namespace DB; + + uintptr_t uintptr = reinterpret_cast(ptr); + ReadBufferFromFile in("/proc/self/maps"); + + while (!in.eof()) + { + uintptr_t begin = readAddressHex(in); + assertChar('-', in); + uintptr_t end = readAddressHex(in); + skipToNextLineOrEOF(in); + + if (begin <= uintptr && uintptr < end) + return {reinterpret_cast(begin), end - begin}; + } + + throw Exception("Cannot find mapped area for pointer", ErrorCodes::LOGICAL_ERROR); +} + +} + +#else + +namespace DB +{ + +std::pair getMappedArea(void * ptr) +{ + throw Exception("The function getMappedArea is implemented only for Linux", ErrorCodes::NOT_IMPLEMENTED); +} + +} + +#endif + diff --git a/src/Common/getMappedArea.h b/src/Common/getMappedArea.h new file mode 100644 index 00000000000..7d120f0d9a7 --- /dev/null +++ b/src/Common/getMappedArea.h @@ -0,0 +1,10 @@ +#include + +namespace DB +{ + +/// Find the address and size of the mapped memory region pointed by ptr. +/// Throw exception if not found. +std::pair getMappedArea(void * ptr); + +} diff --git a/src/Common/remapExecutable.cpp b/src/Common/remapExecutable.cpp index 35d0338f9ce..5418290b24f 100644 --- a/src/Common/remapExecutable.cpp +++ b/src/Common/remapExecutable.cpp @@ -2,17 +2,14 @@ #include #include +#include #include #include -#include - -#include -#include +#include #include -#include -#include +#include #include "remapExecutable.h" @@ -22,7 +19,6 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int CANNOT_ALLOCATE_MEMORY; } @@ -30,24 +26,6 @@ namespace ErrorCodes namespace { -uintptr_t readAddressHex(DB::ReadBuffer & in) -{ - uintptr_t res = 0; - while (!in.eof()) - { - if (isHexDigit(*in.position())) - { - res *= 16; - res += unhex(*in.position()); - ++in.position(); - } - else - break; - } - return res; -} - - __attribute__((__noinline__)) int64_t our_syscall(...) { __asm__ __volatile__ (R"( @@ -157,28 +135,6 @@ __attribute__((__noinline__)) void remapToHugeStep1(void * begin, size_t size) } -std::pair getMappedArea(void * ptr) -{ - using namespace DB; - - uintptr_t uintptr = reinterpret_cast(ptr); - ReadBufferFromFile in("/proc/self/maps"); - - while (!in.eof()) - { - uintptr_t begin = readAddressHex(in); - assertChar('-', in); - uintptr_t end = readAddressHex(in); - skipToNextLineOrEOF(in); - - if (begin <= uintptr && uintptr < end) - return {reinterpret_cast(begin), end - begin}; - } - - throw Exception("Cannot find mapped area for pointer", ErrorCodes::LOGICAL_ERROR); -} - - void remapExecutable() { auto [begin, size] = getMappedArea(reinterpret_cast(remapExecutable)); @@ -193,7 +149,6 @@ namespace DB { void remapExecutable() {} -std::pair getMappedArea(void * ptr) { return {}; } } diff --git a/src/Common/remapExecutable.h b/src/Common/remapExecutable.h index af0a96d5248..7acb61f13bd 100644 --- a/src/Common/remapExecutable.h +++ b/src/Common/remapExecutable.h @@ -4,7 +4,4 @@ namespace DB /// This function tries to reallocate the code of the running program in a more efficient way. void remapExecutable(); -/// Find the address and size of the mapped memory region pointed by ptr. -std::pair getMappedArea(void * ptr); - } From 22bd22702e7b68e0182c81abc6e4a2820f8a1132 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 17 Sep 2020 15:59:14 +0200 Subject: [PATCH 144/273] 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 145/273] 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 146/273] 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 147/273] 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 148/273] 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 149/273] 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 150/273] 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 06ff6d2eda5f0dfc12c6fb13101aa062a0e15ede Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 17 Sep 2020 18:54:41 +0300 Subject: [PATCH 151/273] better 01193_metadata_loading --- .../01193_metadata_loading.reference | 4 +- .../0_stateless/01193_metadata_loading.sh | 38 +++++++++---------- 2 files changed, 20 insertions(+), 22 deletions(-) diff --git a/tests/queries/0_stateless/01193_metadata_loading.reference b/tests/queries/0_stateless/01193_metadata_loading.reference index 8ff246325ac..9789cbf33ba 100644 --- a/tests/queries/0_stateless/01193_metadata_loading.reference +++ b/tests/queries/0_stateless/01193_metadata_loading.reference @@ -1,5 +1,5 @@ -10000 0 2020-06-25 hello [1,2] [3,4] -10000 1 2020-06-26 word [10,20] [30,40] +1000 0 2020-06-25 hello [1,2] [3,4] +1000 1 2020-06-26 word [10,20] [30,40] ok 8000 0 2020-06-25 hello [1,2] [3,4] 8000 1 2020-06-26 word [10,20] [30,40] diff --git a/tests/queries/0_stateless/01193_metadata_loading.sh b/tests/queries/0_stateless/01193_metadata_loading.sh index c16726209a3..0ee583a7265 100755 --- a/tests/queries/0_stateless/01193_metadata_loading.sh +++ b/tests/queries/0_stateless/01193_metadata_loading.sh @@ -6,14 +6,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # it is the worst way of making performance test, nevertheless it can detect significant slowdown and some other issues, that usually found by stress test db="test_01193_$RANDOM" - -declare -A engines -engines[0]="Memory" -engines[1]="File(CSV)" -engines[2]="Log" -engines[3]="StripeLog" -engines[4]="MergeTree ORDER BY i" - tables=1000 threads=10 count_multiplier=1 @@ -24,11 +16,17 @@ debug_or_sanitizer_build=$($CLICKHOUSE_CLIENT -q "WITH ((SELECT value FROM syste if [[ debug_or_sanitizer_build -eq 1 ]]; then tables=100; count_multiplier=10; max_time_ms=1500; fi create_tables() { - for i in $(seq 1 $tables); do - engine=${engines[$((i % ${#engines[@]}))]} - $CLICKHOUSE_CLIENT -q "CREATE TABLE $db.table_$1_$i (i UInt64, d Date, s String, n Nested(i UInt8, f Float32)) ENGINE=$engine" - $CLICKHOUSE_CLIENT -q "INSERT INTO $db.table_$1_$i VALUES (0, '2020-06-25', 'hello', [1, 2], [3, 4]), (1, '2020-06-26', 'word', [10, 20], [30, 40])" - done + $CLICKHOUSE_CLIENT -q "WITH + 'CREATE TABLE $db.table_$1_' AS create1, + ' (i UInt64, d Date, s String, n Nested(i UInt8, f Float32)) ENGINE=' AS create2, + ['Memory', 'File(CSV)', 'Log', 'StripeLog', 'MergeTree ORDER BY i'] AS engines, + 'INSERT INTO $db.table_$1_' AS insert1, + ' VALUES (0, ''2020-06-25'', ''hello'', [1, 2], [3, 4]), (1, ''2020-06-26'', ''word'', [10, 20], [30, 40])' AS insert2 + SELECT arrayStringConcat( + groupArray( + create1 || toString(number) || create2 || engines[1 + number % length(engines)] || ';\n' || + insert1 || toString(number) || insert2 + ), ';\n') FROM numbers($tables) FORMAT TSVRaw;" | $CLICKHOUSE_CLIENT -nm } $CLICKHOUSE_CLIENT -q "CREATE DATABASE $db" @@ -39,15 +37,15 @@ done wait $CLICKHOUSE_CLIENT -q "CREATE TABLE $db.table_merge (i UInt64, d Date, s String, n Nested(i UInt8, f Float32)) ENGINE=Merge('$db', '^table_')" -$CLICKHOUSE_CLIENT -q "SELECT count() * $count_multiplier, i, d, s, n.i, n.f FROM $db.table_merge GROUP BY i, d, s, n.i, n.f ORDER BY i" +$CLICKHOUSE_CLIENT -q "SELECT count() * $count_multiplier, i, d, s, n.i, n.f FROM merge('$db', '^table_9') GROUP BY i, d, s, n.i, n.f ORDER BY i" -$CLICKHOUSE_CLIENT -q "DETACH DATABASE $db" +for i in {1..5}; do + $CLICKHOUSE_CLIENT -q "DETACH DATABASE $db" + $CLICKHOUSE_CLIENT -q "ATTACH DATABASE $db" --query_id="$db-$i"; +done -# get real time, grep seconds, remove point, remove leading zeros -elapsed_ms=$({ time $CLICKHOUSE_CLIENT -q "ATTACH DATABASE $db"; } 2>&1 | grep real | grep -Po "0m\K[0-9\.]*" | tr -d '.' | sed "s/^0*//") -$CLICKHOUSE_CLIENT -q "SELECT '01193_metadata_loading', $elapsed_ms FORMAT Null" # it will be printed to server log - -if [[ $elapsed_ms -le $max_time_ms ]]; then echo ok; fi +$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" +$CLICKHOUSE_CLIENT -q "SELECT if(quantile(0.5)(query_duration_ms) < $max_time_ms, 'ok', toString(groupArray(query_duration_ms))) FROM system.query_log WHERE query_id LIKE '$db-%' AND type=2" $CLICKHOUSE_CLIENT -q "SELECT count() * $count_multiplier, i, d, s, n.i, n.f FROM $db.table_merge GROUP BY i, d, s, n.i, n.f ORDER BY i" From 20d07ed405239592b06f24bd6f965c3651525dc8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 21:04:11 +0300 Subject: [PATCH 152/273] Update comment --- src/Common/ThreadFuzzer.h | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Common/ThreadFuzzer.h b/src/Common/ThreadFuzzer.h index d0693945cb0..dabf6209f67 100644 --- a/src/Common/ThreadFuzzer.h +++ b/src/Common/ThreadFuzzer.h @@ -31,10 +31,8 @@ namespace DB * * Notes: * - it can be also implemented with instrumentation (example: LLVM Xray) instead of signals. - * - it's also reasonable to insert glitches around interesting functions (example: mutex lock/unlock, starting of threads, etc.), - * it is doable with wrapping these functions (todo). * - we should also make the sleep time random. - * - sleep obviously helps, but the effect of yield and migration is unclear. + * - sleep and migration obviously helps, but the effect of yield is unclear. * * In addition, we allow to inject glitches around thread synchronization functions. * Example: From c9baceb7605799dca12ed9450fc2ec230baed523 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 17 Sep 2020 21:07:14 +0300 Subject: [PATCH 153/273] Change thread name for DDLWorker executor threads --- src/Interpreters/DDLWorker.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 526f15d921f..5b346eec54a 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -515,6 +515,7 @@ void DDLWorker::scheduleTasks() { worker_pool.scheduleOrThrowOnError([this, task_ptr = task.release()]() { + setThreadName("DDLWorkerExec"); enqueueTask(DDLTaskPtr(task_ptr)); }); } From a783fe110f0e201bb71a934b7680cd09d219e562 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 21:15:19 +0300 Subject: [PATCH 154/273] Fix build --- src/Common/getMappedArea.cpp | 4 ++-- src/Common/getMappedArea.h | 2 ++ 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Common/getMappedArea.cpp b/src/Common/getMappedArea.cpp index 6817bea8cb1..018db16038b 100644 --- a/src/Common/getMappedArea.cpp +++ b/src/Common/getMappedArea.cpp @@ -1,3 +1,5 @@ +#include "getMappedArea.h" + #if defined(__linux__) #include @@ -6,8 +8,6 @@ #include #include -#include "getMappedArea.h" - namespace DB { diff --git a/src/Common/getMappedArea.h b/src/Common/getMappedArea.h index 7d120f0d9a7..3317f72bdc9 100644 --- a/src/Common/getMappedArea.h +++ b/src/Common/getMappedArea.h @@ -1,4 +1,6 @@ #include +#include + namespace DB { From 0275202167945b28f90a5ec84c25fcf3596766ac Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 17 Sep 2020 21:31:39 +0300 Subject: [PATCH 155/273] 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 0597ad315079bc39da20a8296503e717904d46db Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 17 Sep 2020 21:47:21 +0300 Subject: [PATCH 156/273] fix typo --- src/Databases/DatabaseAtomic.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 8f4a4522c59..e0f72436422 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -302,7 +302,7 @@ void DatabaseAtomic::assertDetachedTableNotInUse(const UUID & uuid) /// To avoid it, we remember UUIDs of detached tables and does not allow ATTACH table with such UUID until detached instance still in use. if (detached_tables.count(uuid)) throw Exception("Cannot attach table with UUID " + toString(uuid) + - ", because it was detached but still used by come query. Retry later.", ErrorCodes::TABLE_ALREADY_EXISTS); + ", because it was detached but still used by some query. Retry later.", ErrorCodes::TABLE_ALREADY_EXISTS); } DatabaseAtomic::DetachedTables DatabaseAtomic::cleenupDetachedTables() From dc677b93fb2e803c6a369c66e960cf9308446f04 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 17 Sep 2020 22:30:17 +0300 Subject: [PATCH 157/273] 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 158/273] 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 7fb4dfea2c625497a13417a8500c3dc58dc1784b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 22:50:43 +0300 Subject: [PATCH 159/273] Small improvements for IStorage::rename --- src/Storages/StorageDistributed.cpp | 4 ++-- src/Storages/StorageFile.cpp | 5 ++++- src/Storages/StorageLog.cpp | 17 ++++++++++------- src/Storages/StorageStripeLog.cpp | 11 +++++++---- src/Storages/StorageTinyLog.cpp | 15 +++++++++------ 5 files changed, 32 insertions(+), 20 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 596681002ab..073dd7cc754 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -866,7 +866,7 @@ void StorageDistributed::flushClusterNodesAllData() void StorageDistributed::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { - if (!relative_data_path.empty()) + if (!relative_data_path.empty() && relative_data_path != new_path_to_table_data) renameOnDisk(new_path_to_table_data); renameInMemory(new_table_id); } @@ -878,7 +878,7 @@ void StorageDistributed::renameOnDisk(const String & new_path_to_table_data) { const String path(disk->getPath()); auto new_path = path + new_path_to_table_data; - Poco::File(path + relative_data_path).renameTo(new_path); + disk->moveDirectory(path + relative_data_path, new_path); LOG_DEBUG(log, "Updating path to {}", new_path); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 9e60d5bad15..cc47047dc78 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -525,9 +525,12 @@ void StorageFile::rename(const String & new_path_to_table_data, const StorageID if (paths.size() != 1) throw Exception("Can't rename table " + getStorageID().getNameForLogs() + " in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); + std::string path_new = getTablePath(base_path + new_path_to_table_data, format_name); + if (path_new == paths[0]) + return; + std::unique_lock lock(rwlock); - std::string path_new = getTablePath(base_path + new_path_to_table_data, format_name); Poco::File(Poco::Path(path_new).parent()).createDirectories(); Poco::File(paths[0]).renameTo(path_new); diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 45ab3293723..8c76085e50c 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -548,17 +548,20 @@ void StorageLog::loadMarks() void StorageLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { - std::unique_lock lock(rwlock); + if (table_path != new_path_to_table_data) + { + std::unique_lock lock(rwlock); - disk->moveDirectory(table_path, new_path_to_table_data); + disk->moveDirectory(table_path, new_path_to_table_data); - table_path = new_path_to_table_data; - file_checker.setPath(table_path + "sizes.json"); + table_path = new_path_to_table_data; + file_checker.setPath(table_path + "sizes.json"); - for (auto & file : files) - file.second.data_file_path = table_path + fileName(file.second.data_file_path); + for (auto & file : files) + file.second.data_file_path = table_path + fileName(file.second.data_file_path); - marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME; + marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME; + } renameInMemory(new_table_id); } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 3e7fb5b853e..c2bbb4e4bd8 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -282,12 +282,15 @@ StorageStripeLog::StorageStripeLog( void StorageStripeLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { - std::unique_lock lock(rwlock); + if (table_path != new_path_to_table_data) + { + std::unique_lock lock(rwlock); - disk->moveDirectory(table_path, new_path_to_table_data); + disk->moveDirectory(table_path, new_path_to_table_data); - table_path = new_path_to_table_data; - file_checker.setPath(table_path + "sizes.json"); + table_path = new_path_to_table_data; + file_checker.setPath(table_path + "sizes.json"); + } renameInMemory(new_table_id); } diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index e7fe7e2d5f9..19b6d52b012 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -407,15 +407,18 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type void StorageTinyLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { - std::unique_lock lock(rwlock); + if (table_path != new_path_to_table_data) + { + std::unique_lock lock(rwlock); - disk->moveDirectory(table_path, new_path_to_table_data); + disk->moveDirectory(table_path, new_path_to_table_data); - table_path = new_path_to_table_data; - file_checker.setPath(table_path + "sizes.json"); + table_path = new_path_to_table_data; + file_checker.setPath(table_path + "sizes.json"); - for (auto & file : files) - file.second.data_file_path = table_path + fileName(file.second.data_file_path); + for (auto & file : files) + file.second.data_file_path = table_path + fileName(file.second.data_file_path); + } renameInMemory(new_table_id); } From ddca262fe63445d9ea077b773926692b25356880 Mon Sep 17 00:00:00 2001 From: roman Date: Thu, 17 Sep 2020 20:54:21 +0100 Subject: [PATCH 160/273] 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 1752bbc1edbebd69369e822dfbf3a544235387e5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Sep 2020 01:29:15 +0300 Subject: [PATCH 161/273] Fix "Arcadia" build --- src/Columns/ya.make | 2 ++ src/Common/ya.make | 2 ++ 2 files changed, 4 insertions(+) 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 72f1fa42756..9e4f4cd76c0 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -50,6 +50,7 @@ SRCS( formatIPv6.cpp formatReadable.cpp getExecutablePath.cpp + getMappedArea.cpp getMultipleKeysFromConfig.cpp getNumberOfPhysicalCPUCores.cpp hasLinuxCapability.cpp @@ -87,6 +88,7 @@ SRCS( StatusFile.cpp StatusInfo.cpp Stopwatch.cpp + StringRef.cpp StringUtils/StringUtils.cpp StudentTTest.cpp SymbolIndex.cpp From 0ad44b821e6f8417075d26700bbea576cefd272c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Sep 2020 01:45:12 +0300 Subject: [PATCH 162/273] 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 d1e90e0e4bbb3d273ae06554991a36613e3844cd Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 18 Sep 2020 03:05:51 +0300 Subject: [PATCH 163/273] fix tests --- .../test_adaptive_granularity/test.py | 14 ----------- tests/integration/test_ttl_replicated/test.py | 25 +++---------------- 2 files changed, 3 insertions(+), 36 deletions(-) diff --git a/tests/integration/test_adaptive_granularity/test.py b/tests/integration/test_adaptive_granularity/test.py index 524c8215081..ec3169bb995 100644 --- a/tests/integration/test_adaptive_granularity/test.py +++ b/tests/integration/test_adaptive_granularity/test.py @@ -274,7 +274,6 @@ def test_mixed_granularity_single_node(start_dynamic_cluster, node): "INSERT INTO table_with_default_granularity VALUES (toDate('2018-09-01'), 1, 333), (toDate('2018-09-02'), 2, 444)") def callback(n): -<<<<<<< HEAD new_config = """ 1 @@ -283,12 +282,6 @@ def test_mixed_granularity_single_node(start_dynamic_cluster, node): n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml", new_config) n.replace_config("/etc/clickhouse-server/config.d/merge_tree_settings.xml", new_config) -======= - n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml", - "1") - n.replace_config("/etc/clickhouse-server/config.d/merge_tree_settings.xml", - "1") ->>>>>>> upstream/master node.restart_with_latest_version(callback_onstop=callback) node.query("SYSTEM RELOAD CONFIG") @@ -331,7 +324,6 @@ def test_version_update_two_nodes(start_dynamic_cluster): assert node12.query("SELECT COUNT() FROM table_with_default_granularity") == '2\n' def callback(n): -<<<<<<< HEAD new_config = """ 0 @@ -340,12 +332,6 @@ def test_version_update_two_nodes(start_dynamic_cluster): n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml", new_config) n.replace_config("/etc/clickhouse-server/config.d/merge_tree_settings.xml", new_config) -======= - n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml", - "0") - n.replace_config("/etc/clickhouse-server/config.d/merge_tree_settings.xml", - "0") ->>>>>>> upstream/master node12.restart_with_latest_version(callback_onstop=callback) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index b8832126b0b..878db2da11f 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -55,20 +55,12 @@ def test_merge_with_ttl_timeout(started_cluster): drop_table([node1, node2], table) for node in [node1, node2]: node.query( -<<<<<<< HEAD - ''' - CREATE TABLE {table}(date DateTime, id UInt32, a Int32 TTL date + INTERVAL 1 DAY, b Int32 TTL date + INTERVAL 1 MONTH) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{table}', '{replica}') - ORDER BY id PARTITION BY toDayOfMonth(date) - SETTINGS min_bytes_for_wide_part=0; - '''.format(replica=node.name, table=table)) -======= ''' CREATE TABLE {table}(date DateTime, id UInt32, a Int32 TTL date + INTERVAL 1 DAY, b Int32 TTL date + INTERVAL 1 MONTH) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{table}', '{replica}') - ORDER BY id PARTITION BY toDayOfMonth(date); + ORDER BY id PARTITION BY toDayOfMonth(date) + SETTINGS min_bytes_for_wide_part=0; '''.format(replica=node.name, table=table)) ->>>>>>> upstream/master node1.query("SYSTEM STOP TTL MERGES {table}".format(table=table)) node2.query("SYSTEM STOP TTL MERGES {table}".format(table=table)) @@ -256,17 +248,6 @@ limitations under the License.""" time.sleep(0.5) node1.query( -<<<<<<< HEAD - """ - CREATE TABLE {name} ( - s1 String, - d1 DateTime - ) ENGINE = {engine} - ORDER BY tuple() - TTL d1 + INTERVAL 1 DAY DELETE - SETTINGS min_bytes_for_wide_part=0 - """.format(name=name, engine=engine)) -======= """ CREATE TABLE {name} ( s1 String, @@ -274,8 +255,8 @@ limitations under the License.""" ) ENGINE = {engine} ORDER BY tuple() TTL d1 + INTERVAL 1 DAY DELETE + SETTINGS min_bytes_for_wide_part=0 """.format(name=name, engine=engine)) ->>>>>>> upstream/master node1.query("""ALTER TABLE {name} MODIFY COLUMN s1 String TTL d1 + INTERVAL 1 SECOND""".format(name=name)) node1.query("""ALTER TABLE {name} ADD COLUMN b1 Int32""".format(name=name)) From 848664c4afed4957fde0f22e88c63ac2b133742a Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 18 Sep 2020 09:34:51 +0800 Subject: [PATCH 164/273] 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 165/273] 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 166/273] 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 e6914acb3b9b93d26c38d7bac5634fd97f14c37b Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Sep 2020 10:44:17 +0300 Subject: [PATCH 167/273] Fix check for merge tree settings on server startup --- src/Storages/MergeTree/MergeTreeSettings.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 7f537ec330a..69a7b6d4eb9 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -77,24 +77,24 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def) void MergeTreeSettings::sanityCheck(const Settings & query_settings) const { - if (number_of_free_entries_in_pool_to_execute_mutation >= query_settings.background_pool_size) + if (number_of_free_entries_in_pool_to_execute_mutation > query_settings.background_pool_size) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "The value of 'number_of_free_entries_in_pool_to_execute_mutation' setting" " ({}) (default values are defined in section of config.xml" " or the value can be specified per table in SETTINGS section of CREATE TABLE query)" - " is greater or equals to the value of 'background_pool_size'" + " is greater the value of 'background_pool_size'" " ({}) (the value is defined in users.xml for default profile)." " This indicates incorrect configuration because mutations cannot work with these settings.", number_of_free_entries_in_pool_to_execute_mutation, query_settings.background_pool_size); } - if (number_of_free_entries_in_pool_to_lower_max_size_of_merge >= query_settings.background_pool_size) + if (number_of_free_entries_in_pool_to_lower_max_size_of_merge > query_settings.background_pool_size) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "The value of 'number_of_free_entries_in_pool_to_lower_max_size_of_merge' setting" " ({}) (default values are defined in section of config.xml" " or the value can be specified per table in SETTINGS section of CREATE TABLE query)" - " is greater or equals to the value of 'background_pool_size'" + " is greater the value of 'background_pool_size'" " ({}) (the value is defined in users.xml for default profile)." " This indicates incorrect configuration because the maximum size of merge will be always lowered.", number_of_free_entries_in_pool_to_lower_max_size_of_merge, From 96a693baec89f8feb7589c96dfee950481887f1a Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Sep 2020 10:46:23 +0300 Subject: [PATCH 168/273] Better message --- src/Storages/MergeTree/MergeTreeSettings.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 69a7b6d4eb9..00580c8d8bb 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -82,7 +82,7 @@ void MergeTreeSettings::sanityCheck(const Settings & query_settings) const throw Exception(ErrorCodes::BAD_ARGUMENTS, "The value of 'number_of_free_entries_in_pool_to_execute_mutation' setting" " ({}) (default values are defined in section of config.xml" " or the value can be specified per table in SETTINGS section of CREATE TABLE query)" - " is greater the value of 'background_pool_size'" + " is greater than the value of 'background_pool_size'" " ({}) (the value is defined in users.xml for default profile)." " This indicates incorrect configuration because mutations cannot work with these settings.", number_of_free_entries_in_pool_to_execute_mutation, @@ -94,7 +94,7 @@ void MergeTreeSettings::sanityCheck(const Settings & query_settings) const throw Exception(ErrorCodes::BAD_ARGUMENTS, "The value of 'number_of_free_entries_in_pool_to_lower_max_size_of_merge' setting" " ({}) (default values are defined in section of config.xml" " or the value can be specified per table in SETTINGS section of CREATE TABLE query)" - " is greater the value of 'background_pool_size'" + " is greater than the value of 'background_pool_size'" " ({}) (the value is defined in users.xml for default profile)." " This indicates incorrect configuration because the maximum size of merge will be always lowered.", number_of_free_entries_in_pool_to_lower_max_size_of_merge, From c8a5d9e0a2da0a30b2f5881a727126a2a5ac9530 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Sep 2020 10:59:26 +0300 Subject: [PATCH 169/273] 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 170/273] 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 171/273] 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 449189dcdacaad381fb901a172d90912afdcb6a3 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 18 Sep 2020 13:47:09 +0300 Subject: [PATCH 172/273] Initialize global thread pool before we fetch configs from ZK --- programs/server/Server.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index aa947b22593..c158a947ca4 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -280,6 +280,11 @@ int Server::main(const std::vector & /*args*/) global_context->makeGlobalContext(); global_context->setApplicationType(Context::ApplicationType::SERVER); + // Initialize global thread pool. Do it before we fetch configs from zookeeper + // nodes (`from_zk`), because ZooKeeper interface uses the pool. We will + // ignore `max_thread_pool_size` in configs we fetch from ZK, but oh well. + GlobalThreadPool::initialize(config().getUInt("max_thread_pool_size", 10000)); + bool has_zookeeper = config().has("zookeeper"); zkutil::ZooKeeperNodeCache main_config_zk_node_cache([&] { return global_context->getZooKeeper(); }); @@ -414,9 +419,6 @@ int Server::main(const std::vector & /*args*/) DateLUT::instance(); LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone()); - /// Initialize global thread pool - GlobalThreadPool::initialize(config().getUInt("max_thread_pool_size", 10000)); - /// Storage with temporary data for processing of heavy queries. { std::string tmp_path = config().getString("tmp_path", path + "tmp/"); From e1cf60990a9d4801a184c4a7b5561016a19f23fd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 18 Sep 2020 14:06:30 +0300 Subject: [PATCH 173/273] 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 174/273] 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 175/273] 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 176/273] 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 1ee2e3d2b398546395b0aac5431c98443f48e910 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Fri, 18 Sep 2020 16:03:48 +0300 Subject: [PATCH 177/273] Review fix --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index f2010b4e34e..2d1a1ae3a09 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1498,7 +1498,9 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( } else { - // For the case of one continuous range of keys we use binary search algorithm + /// In case when SELECT's predicate defines a single continuous interval of keys, + /// we can use binary search algorithm to find the left and right endpoint key marks of such interval. + /// The returned value is the minumum range of marks, containing all keys for which KeyCondition holds LOG_TRACE(log, "Running binary search on index range for part {} ({} marks)", part->name, marks_count); @@ -1536,8 +1538,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( result_range.end = searched_right; LOG_TRACE(log, "Found (RIGHT) boundary mark: {}", searched_right); - - if (may_be_true_in_range(result_range)) + if (result_range.begin < result_range.end && may_be_true_in_range(result_range)) res.emplace_back(std::move(result_range)); LOG_TRACE(log, "Found {} range in {} steps", res.empty() ? "empty" : "continuous", steps); From e5dfc38bfe0918cd5a7500bee2308e8c10ff274e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 18 Sep 2020 16:13:04 +0300 Subject: [PATCH 178/273] 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 179/273] 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 e7b9319e120147735019f3a89a87b45087f86815 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 18 Sep 2020 16:27:50 +0300 Subject: [PATCH 180/273] If perf test definition changed, run everything + longer (as in master) Also some other perf test fixes --- docker/test/performance-comparison/compare.sh | 75 +++++++++++++------ .../test/performance-comparison/entrypoint.sh | 11 +-- docker/test/performance-comparison/report.py | 12 +-- tests/performance/columns_hashing.xml | 17 ++--- 4 files changed, 71 insertions(+), 44 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 8d7947b46a5..886dd0b74f6 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -114,8 +114,6 @@ function run_tests # Just check that the script runs at all "$script_dir/perf.py" --help > /dev/null - changed_test_files="" - # Find the directory with test files. if [ -v CHPC_TEST_PATH ] then @@ -130,14 +128,6 @@ function run_tests else # For PRs, use newer test files so we can test these changes. test_prefix=right/performance - - # If only the perf tests were changed in the PR, we will run only these - # tests. The list of changed tests in changed-test.txt is prepared in - # entrypoint.sh from git diffs, because it has the cloned repo. Used - # to use rsync for that but it was really ugly and not always correct - # (e.g. when the reference SHA is really old and has some other - # differences to the tested SHA, besides the one introduced by the PR). - changed_test_files=$(sed "s/tests\/performance/${test_prefix//\//\\/}/" changed-tests.txt) fi # Determine which tests to run. @@ -146,19 +136,26 @@ function run_tests # Run only explicitly specified tests, if any. # shellcheck disable=SC2010 test_files=$(ls "$test_prefix" | grep "$CHPC_TEST_GREP" | xargs -I{} -n1 readlink -f "$test_prefix/{}") - elif [ "$changed_test_files" != "" ] + elif [ "$PR_TO_TEST" -ne 0 ] \ + && [ "$(wc -l < changed-test-definitions.txt)" -gt 0 ] \ + && [ "$(wc -l < changed-test-scripts.txt)" -eq 0 ] \ + && [ "$(wc -l < other-changed-files.txt)" -eq 0 ] then - # Use test files that changed in the PR. - test_files="$changed_test_files" + # If only the perf tests were changed in the PR, we will run only these + # tests. The lists of changed files are prepared in entrypoint.sh because + # it has the repository. + test_files=$(sed "s/tests\/performance/${test_prefix//\//\\/}/" changed-test-definitions.txt) else # The default -- run all tests found in the test dir. 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" ] + # For PRs w/o changes in test definitons and scripts, 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" -ne 0 ] \ + && [ "$(wc -l < changed-test-definitions.txt)" -eq 0 ] \ + && [ "$(wc -l < changed-test-files.txt)" -eq 0 ] then CHPC_RUNS=${CHPC_RUNS:-13} CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-0} @@ -662,6 +659,38 @@ create table test_time engine Memory as from total_client_time_per_query full join queries using (test, query_index) group by test; +create view query_runs as select * from file('analyze/query-runs.tsv', TSV, + 'test text, query_index int, query_id text, version UInt8, time float'); + +-- +-- Guess the number of query runs used for this test. The number is required to +-- calculate and check the average query run time in the report. +-- We have to be careful, because we will encounter: +-- 1) partial queries which run only on one server +-- 2) short queries which run for a much higher number of times +-- 3) some errors that make query run for a different number of times on a +-- particular server. +-- +create view test_runs as + select test, + -- Default to 7 runs if there are only 'short' queries in the test, and + -- we can't determine the number of runs. + if((ceil(medianOrDefaultIf(t.runs, not short), 0) as r) != 0, r, 7) runs + from ( + select + -- The query id is the same for both servers, so no need to divide here. + uniqExact(query_id) runs, + (test, query_index) in + (select * from file('analyze/marked-short-queries.tsv', TSV, + 'test text, query_index int')) + as short, + test, query_index + from query_runs + group by test, query_index + ) t + group by test + ; + 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), @@ -669,11 +698,15 @@ create table test_times_report engine File(TSV, 'report/test-times.tsv') as short_queries, toDecimal64(query_max, 3), toDecimal64(real / queries, 3) avg_real_per_query, - toDecimal64(query_min, 3) + toDecimal64(query_min, 3), + runs from test_time - -- wall clock times are also measured for skipped tests, so don't - -- do full join - left join wall_clock_time_per_test using test + -- wall clock times are also measured for skipped tests, so don't + -- do full join + left join wall_clock_time_per_test + on wall_clock_time_per_test.test = test_time.test + full join test_runs + on test_runs.test = test_time.test order by avg_real_per_query desc; -- report for all queries page, only main metric diff --git a/docker/test/performance-comparison/entrypoint.sh b/docker/test/performance-comparison/entrypoint.sh index 9e9a46a3ce6..ed2e542eadd 100755 --- a/docker/test/performance-comparison/entrypoint.sh +++ b/docker/test/performance-comparison/entrypoint.sh @@ -97,13 +97,10 @@ then # tests for use by compare.sh. Compare to merge base, because master might be # far in the future and have unrelated test changes. base=$(git -C right/ch merge-base pr origin/master) - git -C right/ch diff --name-only "$base" pr | tee changed-tests.txt - if grep -vq '^tests/performance' changed-tests.txt - then - # Have some other changes besides the tests, so truncate the test list, - # meaning, run all tests. - : > changed-tests.txt - fi + git -C right/ch diff --name-only "$base" pr -- . | tee all-changed-files.txt + git -C right/ch diff --name-only "$base" pr -- tests/performance | tee changed-test-definitions.txt + git -C right/ch diff --name-only "$base" pr -- docker/test/performance-comparison | tee changed-test-scripts.txt + git -C right/ch diff --name-only "$base" pr -- :!tests/performance :!docker/test/performance-comparison | tee other-changed-files.txt fi # Set python output encoding so that we can print queries with Russian letters. diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index e9e2ac68c1e..ee67c3c0457 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -457,25 +457,25 @@ if args.report == 'main': return columns = [ - 'Test', #0 + 'Test', #0 'Wall clock time, s', #1 'Total client time, s', #2 - 'Total queries', #3 - 'Ignored short queries', #4 + '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 + # 'Runs' #8 ] text = tableStart('Test times') text += tableHeader(columns) - nominal_runs = 7 # FIXME pass this as an argument - total_runs = (nominal_runs + 1) * 2 # one prewarm run, two servers - allowed_average_run_time = allowed_single_run_time + 60 / total_runs; # some allowance for fill/create queries + allowed_average_run_time = 3.75 # 60 seconds per test at 7 runs attrs = ['' for c in columns] for r in rows: anchor = f'{currentTableAnchor()}.{r[0]}' + total_runs = (int(r[8]) + 1) * 2 # one prewarm run, two servers if float(r[6]) > allowed_average_run_time * total_runs: # FIXME should be 15s max -- investigate parallel_insert slow_average_tests += 1 diff --git a/tests/performance/columns_hashing.xml b/tests/performance/columns_hashing.xml index fb340c20ccd..271fff6e543 100644 --- a/tests/performance/columns_hashing.xml +++ b/tests/performance/columns_hashing.xml @@ -1,15 +1,12 @@ - - columns_hashing - - - test.hits + hits_10m_single + hits_100m_single - - - - - + select sum(UserID + 1 in (select UserID from hits_100m_single)) from hits_100m_single + select sum((UserID + 1, RegionID) in (select UserID, RegionID from hits_10m_single)) from hits_10m_single + select sum(URL in (select URL from hits_10m_single where URL != '')) from hits_10m_single + select sum(MobilePhoneModel in (select MobilePhoneModel from hits_10m_single where MobilePhoneModel != '')) from hits_10m_single + select sum((MobilePhoneModel, UserID + 1) in (select MobilePhoneModel, UserID from hits_100m_single where MobilePhoneModel != '')) from hits_100m_single From 122e34cf19081952c3df8f37e3d639df1fdfdaa3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Sep 2020 17:38:28 +0300 Subject: [PATCH 181/273] 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 72c2201287f0e67af070c41c02361ad01bb9d5aa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Sep 2020 17:39:33 +0300 Subject: [PATCH 182/273] 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 9e4f4cd76c0..9b671e7b9ee 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -88,7 +88,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 183/273] 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 0da19ab46de88f01cea34e7d7fcab057fd6ce2c4 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 18 Sep 2020 18:30:00 +0300 Subject: [PATCH 184/273] Add option to disable ttl move on data part insert --- src/Disks/IVolume.h | 6 ++- src/Disks/VolumeJBOD.cpp | 3 ++ src/Storages/MergeTree/MergeTreeData.cpp | 43 ++++++++++++++---- src/Storages/MergeTree/MergeTreeData.h | 12 ++--- .../MergeTree/MergeTreeDataWriter.cpp | 2 +- .../config.d/storage_configuration.xml | 12 +++++ tests/integration/test_ttl_move/test.py | 45 +++++++++++++++++++ 7 files changed, 107 insertions(+), 16 deletions(-) diff --git a/src/Disks/IVolume.h b/src/Disks/IVolume.h index 5e7f09e1d04..0f38fe4d22e 100644 --- a/src/Disks/IVolume.h +++ b/src/Disks/IVolume.h @@ -36,10 +36,11 @@ using Volumes = std::vector; class IVolume : public Space { public: - IVolume(String name_, Disks disks_, size_t max_data_part_size_ = 0) + IVolume(String name_, Disks disks_, size_t max_data_part_size_ = 0, bool perform_ttl_move_on_insert_ = true) : disks(std::move(disks_)) , name(name_) , max_data_part_size(max_data_part_size_) + , perform_ttl_move_on_insert(perform_ttl_move_on_insert_) { } @@ -70,6 +71,9 @@ protected: public: /// Max size of reservation, zero means unlimited size UInt64 max_data_part_size = 0; + /// Should a new data part be synchronously moved to a volume according to ttl on insert + /// or move this part in background task asynchronously after insert. + bool perform_ttl_move_on_insert; }; /// Reservation for multiple disks at once. Can be used in RAID1 implementation. diff --git a/src/Disks/VolumeJBOD.cpp b/src/Disks/VolumeJBOD.cpp index bf9dcf7f5d8..3ac8a50acfb 100644 --- a/src/Disks/VolumeJBOD.cpp +++ b/src/Disks/VolumeJBOD.cpp @@ -53,6 +53,9 @@ VolumeJBOD::VolumeJBOD( static constexpr UInt64 MIN_PART_SIZE = 8u * 1024u * 1024u; if (max_data_part_size != 0 && max_data_part_size < MIN_PART_SIZE) LOG_WARNING(logger, "Volume {} max_data_part_size is too low ({} < {})", backQuote(name), ReadableSize(max_data_part_size), ReadableSize(MIN_PART_SIZE)); + + /// Default value is 'true' due to backward compatibility. + perform_ttl_move_on_insert = config.getBool(config_prefix + ".perform_ttl_move_on_insert", true); } DiskPtr VolumeJBOD::getDisk(size_t /* index */) const diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 5969ecc5baf..853ea8eabcf 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2961,11 +2961,12 @@ ReservationPtr MergeTreeData::tryReserveSpace(UInt64 expected_size, SpacePtr spa ReservationPtr MergeTreeData::reserveSpacePreferringTTLRules(UInt64 expected_size, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move, - size_t min_volume_index) const + size_t min_volume_index, + bool is_insert) const { expected_size = std::max(RESERVATION_MIN_ESTIMATION_SIZE, expected_size); - ReservationPtr reservation = tryReserveSpacePreferringTTLRules(expected_size, ttl_infos, time_of_move, min_volume_index); + ReservationPtr reservation = tryReserveSpacePreferringTTLRules(expected_size, ttl_infos, time_of_move, min_volume_index, is_insert); return checkAndReturnReservation(expected_size, std::move(reservation)); } @@ -2973,7 +2974,8 @@ ReservationPtr MergeTreeData::reserveSpacePreferringTTLRules(UInt64 expected_siz ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_size, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move, - size_t min_volume_index) const + size_t min_volume_index, + bool is_insert) const { expected_size = std::max(RESERVATION_MIN_ESTIMATION_SIZE, expected_size); @@ -2984,13 +2986,13 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_ if (ttl_entry) { - SpacePtr destination_ptr = getDestinationForTTL(*ttl_entry); + SpacePtr destination_ptr = getDestinationForTTL(*ttl_entry, is_insert); if (!destination_ptr) { if (ttl_entry->destination_type == DataDestinationType::VOLUME) - LOG_WARNING(log, "Would like to reserve space on volume '{}' by TTL rule of table '{}' but volume was not found", ttl_entry->destination_name, log_name); + LOG_WARNING(log, "Would like to reserve space on volume '{}' by TTL rule of table '{}' but volume was not found or rule is not applicable at the moment", ttl_entry->destination_name, log_name); else if (ttl_entry->destination_type == DataDestinationType::DISK) - LOG_WARNING(log, "Would like to reserve space on disk '{}' by TTL rule of table '{}' but disk was not found", ttl_entry->destination_name, log_name); + LOG_WARNING(log, "Would like to reserve space on disk '{}' by TTL rule of table '{}' but disk was not found or rule is not applicable at the moment", ttl_entry->destination_name, log_name); } else { @@ -3010,13 +3012,36 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_ return reservation; } -SpacePtr MergeTreeData::getDestinationForTTL(const TTLDescription & ttl) const +SpacePtr MergeTreeData::getDestinationForTTL(const TTLDescription & ttl, bool is_insert) const { auto policy = getStoragePolicy(); if (ttl.destination_type == DataDestinationType::VOLUME) - return policy->getVolumeByName(ttl.destination_name); + { + auto volume = policy->getVolumeByName(ttl.destination_name); + + if (!volume) + return {}; + + if (is_insert && !volume->perform_ttl_move_on_insert) + return {}; + + return volume; + } else if (ttl.destination_type == DataDestinationType::DISK) - return policy->getDiskByName(ttl.destination_name); + { + auto disk = policy->getDiskByName(ttl.destination_name); + if (!disk) + return {}; + + auto volume = policy->getVolume(policy->getVolumeIndexByDisk(disk)); + if (!volume) + return {}; + + if (is_insert && !volume->perform_ttl_move_on_insert) + return {}; + + return disk; + } else return {}; } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 82f118a4c0f..59628371ac8 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -624,13 +624,15 @@ public: UInt64 expected_size, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move, - size_t min_volume_index = 0) const; + size_t min_volume_index = 0, + bool is_insert = false) const; ReservationPtr tryReserveSpacePreferringTTLRules( UInt64 expected_size, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move, - size_t min_volume_index = 0) const; + size_t min_volume_index = 0, + bool is_insert = false) const; /// Choose disk with max available free space /// Reserves 0 bytes @@ -638,9 +640,9 @@ public: /// Return alter conversions for part which must be applied on fly. AlterConversions getAlterConversionsForPart(const MergeTreeDataPartPtr part) const; - /// Returns destination disk or volume for the TTL rule according to current - /// storage policy - SpacePtr getDestinationForTTL(const TTLDescription & ttl) const; + /// Returns destination disk or volume for the TTL rule according to current storage policy + /// 'is_insert' - is TTL move performed on new data part insert. + SpacePtr getDestinationForTTL(const TTLDescription & ttl, bool is_insert = false) const; /// Checks if given part already belongs destination disk or volume for the /// TTL rule. diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index e5b684a1361..b8de87ecd3a 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -237,7 +237,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); NamesAndTypesList columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames()); - ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, time(nullptr)); + ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, time(nullptr), true); VolumePtr volume = data.getStoragePolicy()->getVolume(0); auto new_data_part = data.createPart( diff --git a/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml b/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml index 47bf9f56cdd..e96bde89ca9 100644 --- a/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml @@ -83,6 +83,18 @@ + + + +
+ jbod1 +
+ + external + false + +
+
diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index ad822bc6545..cd822025609 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1102,3 +1102,48 @@ limitations under the License.""" finally: node1.query("DROP TABLE IF EXISTS {name} NO DELAY".format(name=name)) + + +@pytest.mark.parametrize("name,dest_type,engine", [ + ("mt_test_disabled_ttl_move_on_insert_work", "DISK", "MergeTree()"), + ("mt_test_disabled_ttl_move_on_insert_work", "VOLUME", "MergeTree()"), + ("replicated_mt_test_disabled_ttl_move_on_insert_work", "DISK", "ReplicatedMergeTree('/clickhouse/replicated_test_disabled_ttl_move_on_insert_work', '1')"), + ("replicated_mt_test_disabled_ttl_move_on_insert_work", "VOLUME", "ReplicatedMergeTree('/clickhouse/replicated_test_disabled_ttl_move_on_insert_work', '1')"), +]) +def test_disabled_ttl_move_on_insert(started_cluster, name, dest_type, engine): + try: + node1.query("SYSTEM STOP MOVES") + + node1.query(""" + CREATE TABLE {name} ( + s1 String, + d1 DateTime + ) ENGINE = {engine} + ORDER BY tuple() + TTL d1 TO {dest_type} 'external' + SETTINGS storage_policy='jbod_without_instant_ttl_move' + """.format(name=name, dest_type=dest_type, engine=engine)) + + data = [] # 10MB in total + for i in range(10): + data.append(("'{}'".format(get_random_string(1024 * 1024)), "toDateTime({})".format( + time.time() - 1))) # 1MB row + + node1.query("INSERT INTO {} (s1, d1) VALUES {}".format(name, ",".join(["(" + ",".join(x) + ")" for x in data]))) + + used_disks = get_used_disks_for_table(node1, name) + assert set(used_disks) == "jbod1" + assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == "10" + + node1.query("SYSTEM START MOVES") + time.sleep(3) + + used_disks = get_used_disks_for_table(node1, name) + assert set(used_disks) == "external" + assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == "10" + + finally: + try: + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) + except: + pass \ No newline at end of file From 6b4b5898a79c494a0607211156687197b82d8ffc Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 18 Sep 2020 18:41:14 +0300 Subject: [PATCH 185/273] Renaming. --- src/Storages/MergeTree/MergeTreeData.cpp | 38 ++++++++++--------- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreePartsMover.cpp | 4 +- 3 files changed, 24 insertions(+), 20 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 853ea8eabcf..6d7af9e88e4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -511,7 +511,7 @@ void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_meta { for (const auto & move_ttl : new_table_ttl.move_ttl) { - if (!getDestinationForTTL(move_ttl)) + if (!getDestinationForMoveTTL(move_ttl)) { String message; if (move_ttl.destination_type == DataDestinationType::DISK) @@ -2982,17 +2982,19 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_ auto metadata_snapshot = getInMemoryMetadataPtr(); ReservationPtr reservation; - auto ttl_entry = selectTTLDescriptionForTTLInfos(metadata_snapshot->getMoveTTLs(), ttl_infos.moves_ttl, time_of_move, true); + auto move_ttl_entry = selectTTLDescriptionForTTLInfos(metadata_snapshot->getMoveTTLs(), ttl_infos.moves_ttl, time_of_move, true); - if (ttl_entry) + if (move_ttl_entry) { - SpacePtr destination_ptr = getDestinationForTTL(*ttl_entry, is_insert); + SpacePtr destination_ptr = getDestinationForMoveTTL(*move_ttl_entry, is_insert); if (!destination_ptr) { - if (ttl_entry->destination_type == DataDestinationType::VOLUME) - LOG_WARNING(log, "Would like to reserve space on volume '{}' by TTL rule of table '{}' but volume was not found or rule is not applicable at the moment", ttl_entry->destination_name, log_name); - else if (ttl_entry->destination_type == DataDestinationType::DISK) - LOG_WARNING(log, "Would like to reserve space on disk '{}' by TTL rule of table '{}' but disk was not found or rule is not applicable at the moment", ttl_entry->destination_name, log_name); + if (move_ttl_entry->destination_type == DataDestinationType::VOLUME) + LOG_WARNING(log, "Would like to reserve space on volume '{}' by TTL rule of table '{}' but volume was not found or rule is not applicable at the moment", + move_ttl_entry->destination_name, log_name); + else if (move_ttl_entry->destination_type == DataDestinationType::DISK) + LOG_WARNING(log, "Would like to reserve space on disk '{}' by TTL rule of table '{}' but disk was not found or rule is not applicable at the moment", + move_ttl_entry->destination_name, log_name); } else { @@ -3000,10 +3002,12 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_ if (reservation) return reservation; else - if (ttl_entry->destination_type == DataDestinationType::VOLUME) - LOG_WARNING(log, "Would like to reserve space on volume '{}' by TTL rule of table '{}' but there is not enough space", ttl_entry->destination_name, log_name); - else if (ttl_entry->destination_type == DataDestinationType::DISK) - LOG_WARNING(log, "Would like to reserve space on disk '{}' by TTL rule of table '{}' but there is not enough space", ttl_entry->destination_name, log_name); + if (move_ttl_entry->destination_type == DataDestinationType::VOLUME) + LOG_WARNING(log, "Would like to reserve space on volume '{}' by TTL rule of table '{}' but there is not enough space", + move_ttl_entry->destination_name, log_name); + else if (move_ttl_entry->destination_type == DataDestinationType::DISK) + LOG_WARNING(log, "Would like to reserve space on disk '{}' by TTL rule of table '{}' but there is not enough space", + move_ttl_entry->destination_name, log_name); } } @@ -3012,12 +3016,12 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_ return reservation; } -SpacePtr MergeTreeData::getDestinationForTTL(const TTLDescription & ttl, bool is_insert) const +SpacePtr MergeTreeData::getDestinationForMoveTTL(const TTLDescription & move_ttl, bool is_insert) const { auto policy = getStoragePolicy(); - if (ttl.destination_type == DataDestinationType::VOLUME) + if (move_ttl.destination_type == DataDestinationType::VOLUME) { - auto volume = policy->getVolumeByName(ttl.destination_name); + auto volume = policy->getVolumeByName(move_ttl.destination_name); if (!volume) return {}; @@ -3027,9 +3031,9 @@ SpacePtr MergeTreeData::getDestinationForTTL(const TTLDescription & ttl, bool is return volume; } - else if (ttl.destination_type == DataDestinationType::DISK) + else if (move_ttl.destination_type == DataDestinationType::DISK) { - auto disk = policy->getDiskByName(ttl.destination_name); + auto disk = policy->getDiskByName(move_ttl.destination_name); if (!disk) return {}; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 59628371ac8..0bd8dd9bc0c 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -642,7 +642,7 @@ public: AlterConversions getAlterConversionsForPart(const MergeTreeDataPartPtr part) const; /// Returns destination disk or volume for the TTL rule according to current storage policy /// 'is_insert' - is TTL move performed on new data part insert. - SpacePtr getDestinationForTTL(const TTLDescription & ttl, bool is_insert = false) const; + SpacePtr getDestinationForMoveTTL(const TTLDescription & move_ttl, bool is_insert = false) const; /// Checks if given part already belongs destination disk or volume for the /// TTL rule. diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 586c4393dfb..c5c6a63abc4 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -136,9 +136,9 @@ bool MergeTreePartsMover::selectPartsForMove( ReservationPtr reservation; if (ttl_entry) { - auto destination = data->getDestinationForTTL(*ttl_entry); + auto destination = data->getDestinationForMoveTTL(*ttl_entry); if (destination && !data->isPartInTTLDestination(*ttl_entry, *part)) - reservation = data->tryReserveSpace(part->getBytesOnDisk(), data->getDestinationForTTL(*ttl_entry)); + reservation = data->tryReserveSpace(part->getBytesOnDisk(), data->getDestinationForMoveTTL(*ttl_entry)); } if (reservation) /// Found reservation by TTL rule. From 76483b8ed3b90525567620ce4d2f8620d762bf1f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 18 Sep 2020 19:14:19 +0300 Subject: [PATCH 186/273] 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 28afbafa08c2362a121334294082ba5599c6b753 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Fri, 18 Sep 2020 19:25:20 +0300 Subject: [PATCH 187/273] fix crash in RIGHT or FULL JOIN switch --- src/Interpreters/ExpressionAnalyzer.cpp | 4 +-- src/Interpreters/ExpressionAnalyzer.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 5 +-- src/Interpreters/JoinSwitcher.h | 35 +++++++++++++++++++ .../01476_right_full_join_switch.reference | 9 +++++ .../01476_right_full_join_switch.sql | 26 ++++++++++++++ 6 files changed, 76 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/01476_right_full_join_switch.reference create mode 100644 tests/queries/0_stateless/01476_right_full_join_switch.sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index d9fc44d9b8f..8d67672612c 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1142,8 +1142,8 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (hasJoin()) { /// You may find it strange but we support read_in_order for HashJoin and do not support for MergeJoin. - bool has_delayed_stream = query_analyzer.analyzedJoin().needStreamWithNonJoinedRows(); - join_allow_read_in_order = typeid_cast(join.get()) && !has_delayed_stream; + join_has_delayed_stream = query_analyzer.analyzedJoin().needStreamWithNonJoinedRows(); + join_allow_read_in_order = typeid_cast(join.get()) && !join_has_delayed_stream; } optimize_read_in_order = diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 1cc9d75b19f..cbfebafa439 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -176,6 +176,7 @@ struct ExpressionAnalysisResult bool remove_where_filter = false; bool optimize_read_in_order = false; bool optimize_aggregation_in_order = false; + bool join_has_delayed_stream = false; ExpressionActionsPtr before_array_join; ArrayJoinActionPtr array_join; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index cd2c16813b4..f049376156b 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -25,7 +25,7 @@ #include #include #include -#include +#include #include #include @@ -925,8 +925,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu join_step->setStepDescription("JOIN"); query_plan.addStep(std::move(join_step)); - if (auto stream = join->createStreamWithNonJoinedRows(join_result_sample, settings.max_block_size)) + if (expressions.join_has_delayed_stream) { + auto stream = std::make_shared(*join, join_result_sample, settings.max_block_size); auto source = std::make_shared(std::move(stream)); auto add_non_joined_rows_step = std::make_unique( query_plan.getCurrentDataStream(), std::move(source)); diff --git a/src/Interpreters/JoinSwitcher.h b/src/Interpreters/JoinSwitcher.h index ea9b94546ef..23f5dff54d7 100644 --- a/src/Interpreters/JoinSwitcher.h +++ b/src/Interpreters/JoinSwitcher.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -75,4 +76,38 @@ private: void switchJoin(); }; + +/// Creates NonJoinedBlockInputStream on the first read. Allows to swap join algo before it. +class LazyNonJoinedBlockInputStream : public IBlockInputStream +{ +public: + LazyNonJoinedBlockInputStream(const IJoin & join_, const Block & block, UInt64 max_block_size_) + : join(join_) + , result_sample_block(block) + , max_block_size(max_block_size_) + {} + + String getName() const override { return "LazyNonMergeJoined"; } + Block getHeader() const override { return result_sample_block; } + +protected: + Block readImpl() override + { + if (!stream) + { + stream = join.createStreamWithNonJoinedRows(result_sample_block, max_block_size); + if (!stream) + return {}; + } + + return stream->read(); + } + +private: + BlockInputStreamPtr stream; + const IJoin & join; + Block result_sample_block; + UInt64 max_block_size; +}; + } diff --git a/tests/queries/0_stateless/01476_right_full_join_switch.reference b/tests/queries/0_stateless/01476_right_full_join_switch.reference new file mode 100644 index 00000000000..1f839b86013 --- /dev/null +++ b/tests/queries/0_stateless/01476_right_full_join_switch.reference @@ -0,0 +1,9 @@ +1 l \N LowCardinality(String) Nullable(String) +2 \N LowCardinality(String) Nullable(String) +1 l \N LowCardinality(String) Nullable(String) +2 \N LowCardinality(String) Nullable(String) +- +0 \N Nullable(String) LowCardinality(String) +1 \N l Nullable(String) LowCardinality(String) +0 \N Nullable(String) LowCardinality(String) +1 \N l Nullable(String) LowCardinality(String) diff --git a/tests/queries/0_stateless/01476_right_full_join_switch.sql b/tests/queries/0_stateless/01476_right_full_join_switch.sql new file mode 100644 index 00000000000..5d041843ee2 --- /dev/null +++ b/tests/queries/0_stateless/01476_right_full_join_switch.sql @@ -0,0 +1,26 @@ +SET join_algorithm = 'auto'; +SET max_bytes_in_join = 100; + +DROP TABLE IF EXISTS t; +DROP TABLE IF EXISTS nr; + +CREATE TABLE t (`x` UInt32, `s` LowCardinality(String)) ENGINE = Memory; +CREATE TABLE nr (`x` Nullable(UInt32), `s` Nullable(String)) ENGINE = Memory; + +INSERT INTO t VALUES (1, 'l'); +INSERT INTO nr VALUES (2, NULL); + +SET join_use_nulls = 0; + +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l LEFT JOIN nr AS r USING (x) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l RIGHT JOIN nr AS r USING (x) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l FULL JOIN nr AS r USING (x) ORDER BY t.x; + +SELECT '-'; + +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l LEFT JOIN t AS r USING (x) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l RIGHT JOIN t AS r USING (x) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l FULL JOIN t AS r USING (x) ORDER BY t.x; + +DROP TABLE t; +DROP TABLE nr; From da04a130ed5a33501664df764e319e52673cfbea Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 18 Sep 2020 20:45:30 +0300 Subject: [PATCH 188/273] Add option to disable ttl move on data part insert - minor fixes. --- src/Disks/IVolume.h | 2 +- src/Storages/MergeTree/DataPartsExchange.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 2 +- .../configs/config.d/storage_configuration.xml | 1 + tests/integration/test_ttl_move/test.py | 8 +++++--- 5 files changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Disks/IVolume.h b/src/Disks/IVolume.h index 0f38fe4d22e..eaf3bf1dbd4 100644 --- a/src/Disks/IVolume.h +++ b/src/Disks/IVolume.h @@ -73,7 +73,7 @@ public: UInt64 max_data_part_size = 0; /// Should a new data part be synchronously moved to a volume according to ttl on insert /// or move this part in background task asynchronously after insert. - bool perform_ttl_move_on_insert; + bool perform_ttl_move_on_insert = true; }; /// Reservation for multiple disks at once. Can be used in RAID1 implementation. diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 0e42d267729..f9fb157942a 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -276,7 +276,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( ReadBufferFromString ttl_infos_buffer(ttl_infos_string); assertString("ttl format version: 1\n", ttl_infos_buffer); ttl_infos.read(ttl_infos_buffer); - reservation = data.reserveSpacePreferringTTLRules(sum_files_size, ttl_infos, std::time(nullptr)); + reservation = data.reserveSpacePreferringTTLRules(sum_files_size, ttl_infos, std::time(nullptr), 0, true); } else reservation = data.reserveSpace(sum_files_size); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index b8de87ecd3a..739aff31a06 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -237,7 +237,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); NamesAndTypesList columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames()); - ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, time(nullptr), true); + ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, time(nullptr), 0, true); VolumePtr volume = data.getStoragePolicy()->getVolume(0); auto new_data_part = data.createPart( diff --git a/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml b/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml index e96bde89ca9..98cd8160ac6 100644 --- a/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml @@ -88,6 +88,7 @@
jbod1 + jbod2
external diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index cd822025609..990b2a742b4 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1113,6 +1113,7 @@ limitations under the License.""" def test_disabled_ttl_move_on_insert(started_cluster, name, dest_type, engine): try: node1.query("SYSTEM STOP MOVES") + node2.query("SYSTEM STOP MOVES") node1.query(""" CREATE TABLE {name} ( @@ -1132,18 +1133,19 @@ def test_disabled_ttl_move_on_insert(started_cluster, name, dest_type, engine): node1.query("INSERT INTO {} (s1, d1) VALUES {}".format(name, ",".join(["(" + ",".join(x) + ")" for x in data]))) used_disks = get_used_disks_for_table(node1, name) - assert set(used_disks) == "jbod1" + assert set(used_disks) == {"jbod1"} assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == "10" node1.query("SYSTEM START MOVES") + node2.query("SYSTEM START MOVES") time.sleep(3) used_disks = get_used_disks_for_table(node1, name) - assert set(used_disks) == "external" + assert set(used_disks) == {"external"} assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == "10" finally: try: node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) except: - pass \ No newline at end of file + pass From d842cb704fb6736620e1a8819d00e5cc1d3d3320 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 18 Sep 2020 23:50:28 +0800 Subject: [PATCH 189/273] Allow mutations to work with key constraints. --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 3 +++ .../test_mutations_with_merge_tree/configs/users.xml | 2 ++ tests/integration/test_mutations_with_merge_tree/test.py | 8 ++++---- 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 89631b713ed..48309303f59 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1079,6 +1079,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor auto context_for_reading = context; context_for_reading.setSetting("max_streams_to_max_threads_ratio", 1); context_for_reading.setSetting("max_threads", 1); + /// Allow mutations to work when force_index_by_date or force_primary_key is on. + context_for_reading.setSetting("force_index_by_date", 0); + context_for_reading.setSetting("force_primary_key", 0); MutationCommands commands_for_part; for (const auto & command : commands) diff --git a/tests/integration/test_mutations_with_merge_tree/configs/users.xml b/tests/integration/test_mutations_with_merge_tree/configs/users.xml index e1dd7fb0638..47dea62971b 100644 --- a/tests/integration/test_mutations_with_merge_tree/configs/users.xml +++ b/tests/integration/test_mutations_with_merge_tree/configs/users.xml @@ -3,6 +3,8 @@ 500 + 1 + 1
diff --git a/tests/integration/test_mutations_with_merge_tree/test.py b/tests/integration/test_mutations_with_merge_tree/test.py index 019f8c2ea40..25bc0df8e7c 100644 --- a/tests/integration/test_mutations_with_merge_tree/test.py +++ b/tests/integration/test_mutations_with_merge_tree/test.py @@ -16,7 +16,7 @@ def started_cluster(): instance_test_mutations.query( '''CREATE TABLE test_mutations_with_ast_elements(date Date, a UInt64, b String) ENGINE = MergeTree(date, (a, date), 8192)''') instance_test_mutations.query( - '''INSERT INTO test_mutations_with_ast_elements SELECT '2019-07-29' AS date, 1, toString(number) FROM numbers(1)''') + '''INSERT INTO test_mutations_with_ast_elements SELECT '2019-07-29' AS date, 1, toString(number) FROM numbers(1) SETTINGS force_index_by_date = 0, force_primary_key = 0''') yield cluster finally: cluster.shutdown() @@ -38,14 +38,14 @@ def test_mutations_with_merge_background_task(started_cluster): instance_test_mutations.query('''DETACH TABLE test_mutations_with_ast_elements''') instance_test_mutations.query('''ATTACH TABLE test_mutations_with_ast_elements''') return int(instance.query( - "SELECT sum(is_done) FROM system.mutations WHERE table = 'test_mutations_with_ast_elements'").rstrip()) + "SELECT sum(is_done) FROM system.mutations WHERE table = 'test_mutations_with_ast_elements' SETTINGS force_index_by_date = 0, force_primary_key = 0").rstrip()) if get_done_mutations(instance_test_mutations) == 100: all_done = True break print instance_test_mutations.query( - "SELECT mutation_id, command, parts_to_do, is_done FROM system.mutations WHERE table = 'test_mutations_with_ast_elements' FORMAT TSVWithNames") + "SELECT mutation_id, command, parts_to_do, is_done FROM system.mutations WHERE table = 'test_mutations_with_ast_elements' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames") assert all_done @@ -59,4 +59,4 @@ def test_mutations_with_truncate_table(started_cluster): instance_test_mutations.query("TRUNCATE TABLE test_mutations_with_ast_elements") assert instance_test_mutations.query( - "SELECT COUNT() FROM system.mutations WHERE table = 'test_mutations_with_ast_elements'").rstrip() == '0' + "SELECT COUNT() FROM system.mutations WHERE table = 'test_mutations_with_ast_elements SETTINGS force_index_by_date = 0, force_primary_key = 0'").rstrip() == '0' From 6e6909f47d43b5d14b7550c519dba125b844c00a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Sep 2020 21:54:54 +0300 Subject: [PATCH 190/273] Fix "Arcadia" --- src/Common/getMappedArea.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Common/getMappedArea.cpp b/src/Common/getMappedArea.cpp index 018db16038b..921d95ab740 100644 --- a/src/Common/getMappedArea.cpp +++ b/src/Common/getMappedArea.cpp @@ -15,7 +15,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int NOT_IMPLEMENTED; } @@ -69,7 +68,12 @@ std::pair getMappedArea(void * ptr) namespace DB { -std::pair getMappedArea(void * ptr) +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +std::pair getMappedArea(void *) { throw Exception("The function getMappedArea is implemented only for Linux", ErrorCodes::NOT_IMPLEMENTED); } From 8a8e4f8a41e59e4ce73b823cdf721af02dce92d6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 13:28:54 +0300 Subject: [PATCH 191/273] 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 77be35a2b83dd1427460aee63767346b4e2cee49 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 18 Sep 2020 21:59:56 +0300 Subject: [PATCH 192/273] Fixed test_disabled_ttl_move_on_insert test --- .../configs/config.d/storage_configuration.xml | 1 - tests/integration/test_ttl_move/test.py | 8 +++----- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml b/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml index 98cd8160ac6..e96bde89ca9 100644 --- a/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml @@ -88,7 +88,6 @@
jbod1 - jbod2
external diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 990b2a742b4..5c97e097f1c 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1112,9 +1112,6 @@ limitations under the License.""" ]) def test_disabled_ttl_move_on_insert(started_cluster, name, dest_type, engine): try: - node1.query("SYSTEM STOP MOVES") - node2.query("SYSTEM STOP MOVES") - node1.query(""" CREATE TABLE {name} ( s1 String, @@ -1125,6 +1122,8 @@ def test_disabled_ttl_move_on_insert(started_cluster, name, dest_type, engine): SETTINGS storage_policy='jbod_without_instant_ttl_move' """.format(name=name, dest_type=dest_type, engine=engine)) + node1.query("SYSTEM STOP MOVES {}".format(name)) + data = [] # 10MB in total for i in range(10): data.append(("'{}'".format(get_random_string(1024 * 1024)), "toDateTime({})".format( @@ -1136,8 +1135,7 @@ def test_disabled_ttl_move_on_insert(started_cluster, name, dest_type, engine): assert set(used_disks) == {"jbod1"} assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == "10" - node1.query("SYSTEM START MOVES") - node2.query("SYSTEM START MOVES") + node1.query("SYSTEM START MOVES {}").format(name) time.sleep(3) used_disks = get_used_disks_for_table(node1, name) From 1f429b3d5298e146b51ca77305b3a43786ae0e52 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 17 Sep 2020 13:39:13 +0300 Subject: [PATCH 193/273] 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 194/273] 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 \; From e1ffa07a3983781009da90f208841cc815a96fc4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Sep 2020 22:08:53 +0300 Subject: [PATCH 195/273] Resolve review comments --- src/Storages/StorageDistributed.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 073dd7cc754..e573b264e4f 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -876,10 +876,9 @@ void StorageDistributed::renameOnDisk(const String & new_path_to_table_data) { for (const DiskPtr & disk : data_volume->getDisks()) { - const String path(disk->getPath()); - auto new_path = path + new_path_to_table_data; - disk->moveDirectory(path + relative_data_path, new_path); + disk->moveDirectory(relative_data_path, new_path_to_table_data); + auto new_path = disk->getPath() + new_path_to_table_data; LOG_DEBUG(log, "Updating path to {}", new_path); std::lock_guard lock(cluster_nodes_mutex); From 1fc3aa3ea8ae87d0a184bd32abeb690dd4885ef5 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 18 Sep 2020 22:10:49 +0300 Subject: [PATCH 196/273] Fixed test_disabled_ttl_move_on_insert test --- tests/integration/test_ttl_move/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 5c97e097f1c..377ee0e5d75 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1135,7 +1135,7 @@ def test_disabled_ttl_move_on_insert(started_cluster, name, dest_type, engine): assert set(used_disks) == {"jbod1"} assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == "10" - node1.query("SYSTEM START MOVES {}").format(name) + node1.query("SYSTEM START MOVES {}".format(name)) time.sleep(3) used_disks = get_used_disks_for_table(node1, name) From 24b334258b31418f81ff8cbf8452a33289fb97bf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Sep 2020 22:25:56 +0300 Subject: [PATCH 197/273] Resolve review comment --- src/Storages/StorageDistributed.cpp | 4 +++- src/Storages/StorageLog.cpp | 4 +++- src/Storages/StorageStripeLog.cpp | 4 +++- src/Storages/StorageTinyLog.cpp | 3 ++- 4 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index e573b264e4f..b06434b6317 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -57,6 +57,7 @@ #include #include #include +#include namespace @@ -866,7 +867,8 @@ void StorageDistributed::flushClusterNodesAllData() void StorageDistributed::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { - if (!relative_data_path.empty() && relative_data_path != new_path_to_table_data) + assert(relative_data_path != new_path_to_table_data); + if (!relative_data_path.empty()) renameOnDisk(new_path_to_table_data); renameInMemory(new_table_id); } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 8c76085e50c..e437bfb05f1 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -27,6 +27,8 @@ #include #include +#include + #define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin" #define DBMS_STORAGE_LOG_MARKS_FILE_NAME "__marks.mrk" @@ -548,7 +550,7 @@ void StorageLog::loadMarks() void StorageLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { - if (table_path != new_path_to_table_data) + assert(table_path != new_path_to_table_data); { std::unique_lock lock(rwlock); diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index c2bbb4e4bd8..c4344cf6f1f 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -35,6 +35,8 @@ #include #include +#include + namespace DB { @@ -282,7 +284,7 @@ StorageStripeLog::StorageStripeLog( void StorageStripeLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { - if (table_path != new_path_to_table_data) + assert(table_path != new_path_to_table_data); { std::unique_lock lock(rwlock); diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 19b6d52b012..0bdcab8abf4 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -3,6 +3,7 @@ #include #include +#include #include @@ -407,7 +408,7 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type void StorageTinyLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { - if (table_path != new_path_to_table_data) + assert(table_path != new_path_to_table_data); { std::unique_lock lock(rwlock); From 7d0f3db63957a37b125d76ef04401f3dad17101d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Sep 2020 23:19:41 +0300 Subject: [PATCH 198/273] Fix annoying "Arcadia" --- src/Common/getMappedArea.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/getMappedArea.cpp b/src/Common/getMappedArea.cpp index 921d95ab740..6e452f32b96 100644 --- a/src/Common/getMappedArea.cpp +++ b/src/Common/getMappedArea.cpp @@ -1,10 +1,10 @@ #include "getMappedArea.h" +#include #if defined(__linux__) #include #include -#include #include #include From 79674ea892048dd58b746662a2904af649c87d3f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 19 Sep 2020 00:11:09 +0300 Subject: [PATCH 199/273] Drop python from build dependencies I don't see where it can be required. And plus there is no python package in debian:testing anymore, only: - python2 - python3 So that said that you cannot build debian package on debian:testing anymore (but debian:stable don't have gcc9 for example). Everything would be great, if there will be packages for python2, i.e. python2-lxml, but there isn't, I guess because python2 had been deprecated long time ago... So it looks like converting to python3 is coming. Just for the history, now it is: - debian:stable -> buster - debian:testing -> bullseye --- debian/control | 1 - 1 file changed, 1 deletion(-) diff --git a/debian/control b/debian/control index 58efd711d27..1014b8b0a3c 100644 --- a/debian/control +++ b/debian/control @@ -11,7 +11,6 @@ Build-Depends: debhelper (>= 9), libicu-dev, libreadline-dev, gperf, - python, tzdata Standards-Version: 3.9.8 From dbc837c14858fd5d8921fee59b36d51f4b71d4e0 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 17 Sep 2020 12:58:34 +0300 Subject: [PATCH 200/273] IAccessStorage now can override the login() function. --- src/Access/AccessControlManager.cpp | 5 +++ src/Access/AccessControlManager.h | 1 + src/Access/ContextAccess.cpp | 17 -------- src/Access/ContextAccess.h | 3 -- src/Access/IAccessStorage.cpp | 62 ++++++++++++++++++++++++++++ src/Access/IAccessStorage.h | 17 ++++++++ src/Access/MultipleAccessStorage.cpp | 54 ++++++++++++++++++++++++ src/Access/MultipleAccessStorage.h | 2 + src/Interpreters/Context.cpp | 25 +++++------ src/Interpreters/Context.h | 4 +- 10 files changed, 155 insertions(+), 35 deletions(-) diff --git a/src/Access/AccessControlManager.cpp b/src/Access/AccessControlManager.cpp index 41137867213..dd711e93a95 100644 --- a/src/Access/AccessControlManager.cpp +++ b/src/Access/AccessControlManager.cpp @@ -339,6 +339,11 @@ void AccessControlManager::addStoragesFromMainConfig( } +UUID AccessControlManager::login(const String & user_name, const String & password, const Poco::Net::IPAddress & address) const +{ + return MultipleAccessStorage::login(user_name, password, address, *external_authenticators); +} + void AccessControlManager::setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config) { external_authenticators->setConfig(config, getLogger()); diff --git a/src/Access/AccessControlManager.h b/src/Access/AccessControlManager.h index d7cf59cfb28..81a66ce8f1d 100644 --- a/src/Access/AccessControlManager.h +++ b/src/Access/AccessControlManager.h @@ -106,6 +106,7 @@ public: bool isSettingNameAllowed(const std::string_view & name) const; void checkSettingNameIsAllowed(const std::string_view & name) const; + UUID login(const String & user_name, const String & password, const Poco::Net::IPAddress & address) const; void setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config); std::shared_ptr getContextAccess( diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index d5e48baf110..fbc69e94a43 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -288,23 +288,6 @@ void ContextAccess::calculateAccessRights() const } -bool ContextAccess::isCorrectPassword(const String & password) const -{ - std::lock_guard lock{mutex}; - if (!user) - return false; - return user->authentication.isCorrectPassword(password, user_name, manager->getExternalAuthenticators()); -} - -bool ContextAccess::isClientHostAllowed() const -{ - std::lock_guard lock{mutex}; - if (!user) - return false; - return user->allowed_client_hosts.contains(params.address); -} - - UserPtr ContextAccess::getUser() const { std::lock_guard lock{mutex}; diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 9a5758b79a6..b42d50c3739 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -63,9 +63,6 @@ public: UserPtr getUser() const; String getUserName() const; - bool isCorrectPassword(const String & password) const; - bool isClientHostAllowed() const; - /// Returns information about current and enabled roles. /// The function can return nullptr. std::shared_ptr getRolesInfo() const; diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 874ae612034..e5170221e18 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -13,6 +14,7 @@ namespace ErrorCodes extern const int ACCESS_ENTITY_ALREADY_EXISTS; extern const int ACCESS_ENTITY_NOT_FOUND; extern const int ACCESS_STORAGE_READONLY; + extern const int AUTHENTICATION_FAILED; extern const int LOGICAL_ERROR; } @@ -412,6 +414,57 @@ void IAccessStorage::notify(const Notifications & notifications) } +UUID IAccessStorage::login( + const String & user_name, + const String & password, + const Poco::Net::IPAddress & address, + const ExternalAuthenticators & external_authenticators) const +{ + return loginImpl(user_name, password, address, external_authenticators); +} + + +UUID IAccessStorage::loginImpl( + const String & user_name, + const String & password, + const Poco::Net::IPAddress & address, + const ExternalAuthenticators & external_authenticators) const +{ + if (auto id = find(user_name)) + { + if (auto user = tryRead(*id)) + { + if (isPasswordCorrectImpl(*user, password, external_authenticators) && isAddressAllowedImpl(*user, address)) + return *id; + } + } + throwCannotAuthenticate(user_name); +} + + +bool IAccessStorage::isPasswordCorrectImpl(const User & user, const String & password, const ExternalAuthenticators & external_authenticators) const +{ + return user.authentication.isCorrectPassword(password, user.getName(), external_authenticators); +} + + +bool IAccessStorage::isAddressAllowedImpl(const User & user, const Poco::Net::IPAddress & address) const +{ + return user.allowed_client_hosts.contains(address); +} + +UUID IAccessStorage::getIDOfLoggedUser(const String & user_name) const +{ + return getIDOfLoggedUserImpl(user_name); +} + + +UUID IAccessStorage::getIDOfLoggedUserImpl(const String & user_name) const +{ + return getID(user_name); +} + + UUID IAccessStorage::generateRandomID() { static Poco::UUIDGenerator generator; @@ -500,4 +553,13 @@ void IAccessStorage::throwReadonlyCannotRemove(EntityType type, const String & n "Cannot remove " + outputEntityTypeAndName(type, name) + " from " + getStorageName() + " because this storage is readonly", ErrorCodes::ACCESS_STORAGE_READONLY); } + + +void IAccessStorage::throwCannotAuthenticate(const String & user_name) +{ + /// We use the same message for all authentification failures because we don't want to give away any unnecessary information for security reasons, + /// only the log will show the exact reason. + throw Exception(user_name + ": Authentication failed: password is incorrect or there is no user with such name", ErrorCodes::AUTHENTICATION_FAILED); +} + } diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index 7851f8c9b6b..962ea9427ed 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -11,9 +11,13 @@ namespace Poco { class Logger; } +namespace Poco::Net { class IPAddress; } namespace DB { +struct User; +class ExternalAuthenticators; + /// Contains entities, i.e. instances of classes derived from IAccessEntity. /// The implementations of this class MUST be thread-safe. class IAccessStorage @@ -137,6 +141,14 @@ public: bool hasSubscription(EntityType type) const; bool hasSubscription(const UUID & id) const; + /// Finds an user, check its password and returns the ID of the user. + /// Throws an exception if no such user or password is incorrect. + UUID login(const String & user_name, const String & password, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators) const; + + /// Returns the ID of an user who has logged in (maybe on another node). + /// The function assumes that the password has been already checked somehow, so we can skip checking it now. + UUID getIDOfLoggedUser(const String & user_name) const; + protected: virtual std::optional findImpl(EntityType type, const String & name) const = 0; virtual std::vector findAllImpl(EntityType type) const = 0; @@ -151,6 +163,10 @@ protected: virtual ext::scope_guard subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const = 0; virtual bool hasSubscriptionImpl(const UUID & id) const = 0; virtual bool hasSubscriptionImpl(EntityType type) const = 0; + virtual UUID loginImpl(const String & user_name, const String & password, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators) const; + virtual bool isPasswordCorrectImpl(const User & user, const String & password, const ExternalAuthenticators & external_authenticators) const; + virtual bool isAddressAllowedImpl(const User & user, const Poco::Net::IPAddress & address) const; + virtual UUID getIDOfLoggedUserImpl(const String & user_name) const; static UUID generateRandomID(); Poco::Logger * getLogger() const; @@ -165,6 +181,7 @@ protected: [[noreturn]] void throwReadonlyCannotInsert(EntityType type, const String & name) const; [[noreturn]] void throwReadonlyCannotUpdate(EntityType type, const String & name) const; [[noreturn]] void throwReadonlyCannotRemove(EntityType type, const String & name) const; + [[noreturn]] static void throwCannotAuthenticate(const String & user_name); using Notification = std::tuple; using Notifications = std::vector; diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index bf711b54d54..8ddc7410d8d 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -392,4 +392,58 @@ void MultipleAccessStorage::updateSubscriptionsToNestedStorages(std::unique_lock added_subscriptions->clear(); } + +UUID MultipleAccessStorage::loginImpl(const String & user_name, const String & password, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators) const +{ + auto storages = getStoragesInternal(); + for (const auto & storage : *storages) + { + try + { + auto id = storage->login(user_name, password, address, external_authenticators); + std::lock_guard lock{mutex}; + ids_cache.set(id, storage); + return id; + } + catch (...) + { + if (!storage->find(EntityType::USER, user_name)) + { + /// The authentication failed because there no users with such name in the `storage` + /// thus we can try to search in other nested storages. + continue; + } + throw; + } + } + throwCannotAuthenticate(user_name); +} + + +UUID MultipleAccessStorage::getIDOfLoggedUserImpl(const String & user_name) const +{ + auto storages = getStoragesInternal(); + for (const auto & storage : *storages) + { + try + { + auto id = storage->getIDOfLoggedUser(user_name); + std::lock_guard lock{mutex}; + ids_cache.set(id, storage); + return id; + } + catch (...) + { + if (!storage->find(EntityType::USER, user_name)) + { + /// The authentication failed because there no users with such name in the `storage` + /// thus we can try to search in other nested storages. + continue; + } + throw; + } + } + throwNotFound(EntityType::USER, user_name); +} + } diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index 5d01894621f..36551f1cbc8 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -47,6 +47,8 @@ protected: ext::scope_guard subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const override; bool hasSubscriptionImpl(const UUID & id) const override; bool hasSubscriptionImpl(EntityType type) const override; + UUID loginImpl(const String & user_name, const String & password, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators) const override; + UUID getIDOfLoggedUserImpl(const String & user_name) const override; private: using Storages = std::vector; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4210c3be1fc..d49ac48da80 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -106,7 +106,6 @@ namespace ErrorCodes extern const int SESSION_NOT_FOUND; extern const int SESSION_IS_LOCKED; extern const int LOGICAL_ERROR; - extern const int AUTHENTICATION_FAILED; extern const int NOT_IMPLEMENTED; } @@ -689,22 +688,20 @@ void Context::setUserImpl(const String & name, const std::optional & pas client_info.current_password = password.value_or(""); #endif - auto new_user_id = getAccessControlManager().find(name); - std::shared_ptr new_access; - if (new_user_id) + /// Find a user with such name and check the password. + UUID new_user_id; + if (password) + new_user_id = getAccessControlManager().login(name, *password, address.host()); + else { - new_access = getAccessControlManager().getContextAccess(*new_user_id, {}, true, settings, current_database, client_info); /// Access w/o password is done under interserver-secret (remote_servers.secret) - /// So it is okay not to check client's host (since there is trust). - if (password && (!new_access->isClientHostAllowed() || !new_access->isCorrectPassword(*password))) - { - new_user_id = {}; - new_access = nullptr; - } + /// So it is okay not to check client's host in this case (since there is trust). + new_user_id = getAccessControlManager().getIDOfLoggedUser(name); } - if (!new_user_id || !new_access) - throw Exception(name + ": Authentication failed: password is incorrect or there is no user with such name", ErrorCodes::AUTHENTICATION_FAILED); + auto new_access = getAccessControlManager().getContextAccess( + new_user_id, /* current_roles = */ {}, /* use_default_roles = */ true, + settings, current_database, client_info); user_id = new_user_id; access = std::move(new_access); @@ -716,7 +713,7 @@ void Context::setUserImpl(const String & name, const std::optional & pas void Context::setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address) { - setUserImpl(name, std::make_optional(password), address); + setUserImpl(name, password, address); } void Context::setUserWithoutCheckingPassword(const String & name, const Poco::Net::SocketAddress & address) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b15ea018573..e0bd5606bd2 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -258,11 +258,13 @@ public: /// Sets the current user, checks the password and that the specified host is allowed. /// Must be called before getClientInfo. void setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address); - /// Sets the current user, *do not checks the password* but check that the specified host is allowed. + + /// Sets the current user, *do not checks the password and that the specified host is allowed*. /// Must be called before getClientInfo. /// /// (Used only internally in cluster, if the secret matches) void setUserWithoutCheckingPassword(const String & name, const Poco::Net::SocketAddress & address); + void setQuotaKey(String quota_key_); UserPtr getUser() const; From 02acedd6a32149c39b709c6c298c8767b6f54006 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 17 Sep 2020 17:14:46 +0300 Subject: [PATCH 201/273] Use initial_address for quota calculation when the interserver-secret mode enabled. --- src/Server/TCPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index d3a5ea38f3f..4d77759e517 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -965,7 +965,7 @@ void TCPHandler::receiveQuery() /// i.e. when the INSERT is done with the global context (w/o user). if (!client_info.initial_user.empty()) { - query_context->setUserWithoutCheckingPassword(client_info.initial_user, socket().peerAddress()); + query_context->setUserWithoutCheckingPassword(client_info.initial_user, client_info.initial_address); LOG_DEBUG(log, "User (initial): {}", query_context->getUserName()); } /// No need to update connection_context, since it does not requires user (it will not be used for query execution) From 4e77c2e48e7e4c5e56ae5dfccdc0216433804094 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 19 Sep 2020 01:29:31 +0300 Subject: [PATCH 202/273] Update MergeTreeDataMergerMutator.cpp --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 48309303f59..91a34efc2b5 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1080,8 +1080,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor context_for_reading.setSetting("max_streams_to_max_threads_ratio", 1); context_for_reading.setSetting("max_threads", 1); /// Allow mutations to work when force_index_by_date or force_primary_key is on. - context_for_reading.setSetting("force_index_by_date", 0); - context_for_reading.setSetting("force_primary_key", 0); + context_for_reading.setSetting("force_index_by_date", Field(0)); + context_for_reading.setSetting("force_primary_key", Field(0)); MutationCommands commands_for_part; for (const auto & command : commands) From 37facf44825a35bc688c2de94f1db4644fab67da Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 17:44:46 +0300 Subject: [PATCH 203/273] Fix clang-tidy --- base/common/coverage.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/base/common/coverage.cpp b/base/common/coverage.cpp index 9f3c5ca653a..043f97f9593 100644 --- a/base/common/coverage.cpp +++ b/base/common/coverage.cpp @@ -3,12 +3,11 @@ #if WITH_COVERAGE # include - # include # if defined(__clang__) -extern "C" void __llvm_profile_dump(); +extern "C" void __llvm_profile_dump(); // NOLINT # elif defined(__GNUC__) || defined(__GNUG__) extern "C" void __gcov_exit(); # endif @@ -23,7 +22,7 @@ void dumpCoverageReportIfPossible() std::lock_guard lock(mutex); # if defined(__clang__) - __llvm_profile_dump(); + __llvm_profile_dump(); // NOLINT # elif defined(__GNUC__) || defined(__GNUG__) __gcov_exit(); # endif From ef5363b87f22ab35a52f5705d3b69c6dbc1621aa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 02:04:36 +0300 Subject: [PATCH 204/273] Fix terribly wrong code --- base/common/phdr_cache.cpp | 4 +- programs/server/Server.cpp | 1 - src/CMakeLists.txt | 1 + src/Common/SymbolIndex.cpp | 2 +- src/Common/tests/int_hashes_perf.cpp | 7 +- src/DataTypes/convertMySQLDataType.cpp | 126 ++++---- src/Disks/S3/registerDiskS3.cpp | 1 - src/Functions/appendTrailingCharIfAbsent.cpp | 2 +- src/IO/S3/PocoHTTPClient.cpp | 19 +- src/IO/S3Common.cpp | 2 - ...est_aux_funcs_for_adaptive_granularity.cpp | 2 +- ...for_adaptive_granularity_compact_parts.cpp | 2 +- utils/db-generator/query_db_generator.cpp | 284 +++++++++--------- 13 files changed, 225 insertions(+), 228 deletions(-) diff --git a/base/common/phdr_cache.cpp b/base/common/phdr_cache.cpp index f362fb64285..4f6a066adab 100644 --- a/base/common/phdr_cache.cpp +++ b/base/common/phdr_cache.cpp @@ -14,7 +14,7 @@ # pragma clang diagnostic ignored "-Wunused-macros" #endif -#define __msan_unpoison(X, Y) +#define __msan_unpoison(X, Y) // NOLINT #if defined(__has_feature) # if __has_feature(memory_sanitizer) # undef __msan_unpoison @@ -84,7 +84,7 @@ extern "C" #ifdef ADDRESS_SANITIZER void __lsan_ignore_object(const void *); #else -void __lsan_ignore_object(const void *) {} +void __lsan_ignore_object(const void *) {} // NOLINT #endif } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index aa947b22593..97212e3ddcf 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -43,7 +43,6 @@ #include #include #include -#include #include #include #include diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index b6e8c395b26..8f84939c5a4 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -67,6 +67,7 @@ set(dbms_sources) add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common/HashTable) add_headers_and_sources(clickhouse_common_io IO) +add_headers_and_sources(clickhouse_common_io IO/S3) list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_delete.cpp) if(USE_RDKAFKA) diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index ffa7f0462c9..a738512bb30 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -59,7 +59,7 @@ Otherwise you will get only exported symbols from program headers. # pragma clang diagnostic ignored "-Wunused-macros" #endif -#define __msan_unpoison_string(X) +#define __msan_unpoison_string(X) // NOLINT #if defined(__has_feature) # if __has_feature(memory_sanitizer) # undef __msan_unpoison_string diff --git a/src/Common/tests/int_hashes_perf.cpp b/src/Common/tests/int_hashes_perf.cpp index 7e8495cef27..569e9273bc4 100644 --- a/src/Common/tests/int_hashes_perf.cpp +++ b/src/Common/tests/int_hashes_perf.cpp @@ -187,11 +187,6 @@ static inline size_t tabulation(UInt64 x) return res; } -static inline size_t _intHash64(UInt64 x) -{ - return static_cast(intHash64(x)); -} - const size_t BUF_SIZE = 1024; @@ -284,7 +279,7 @@ int main(int argc, char ** argv) if (!method || method == 1) test (n, data.data(), "0: identity"); if (!method || method == 2) test (n, data.data(), "1: intHash32"); - if (!method || method == 3) test<_intHash64>(n, data.data(), "2: intHash64"); + if (!method || method == 3) test (n, data.data(), "2: intHash64"); if (!method || method == 4) test (n, data.data(), "3: two rounds"); if (!method || method == 5) test (n, data.data(), "4: two rounds and two variables"); if (!method || method == 6) test (n, data.data(), "5: two rounds with less ops"); diff --git a/src/DataTypes/convertMySQLDataType.cpp b/src/DataTypes/convertMySQLDataType.cpp index a509cf8b091..ef6fb7de874 100644 --- a/src/DataTypes/convertMySQLDataType.cpp +++ b/src/DataTypes/convertMySQLDataType.cpp @@ -40,74 +40,76 @@ DataTypePtr convertMySQLDataType(MultiEnum type_support, { // we expect mysql_data_type to be either "basic_type" or "type_with_params(param1, param2, ...)" auto data_type = std::string_view(mysql_data_type); - const auto param_start_pos = data_type.find("("); + const auto param_start_pos = data_type.find('('); const auto type_name = data_type.substr(0, param_start_pos); - DataTypePtr res = [&]() -> DataTypePtr { - if (type_name == "tinyint") - { - if (is_unsigned) - return std::make_shared(); - else - return std::make_shared(); - } - if (type_name == "smallint") - { - if (is_unsigned) - return std::make_shared(); - else - return std::make_shared(); - } - if (type_name == "int" || type_name == "mediumint") - { - if (is_unsigned) - return std::make_shared(); - else - return std::make_shared(); - } - if (type_name == "bigint") - { - if (is_unsigned) - return std::make_shared(); - else - return std::make_shared(); - } - if (type_name == "float") - return std::make_shared(); - if (type_name == "double") - return std::make_shared(); - if (type_name == "date") - return std::make_shared(); - if (type_name == "binary") - return std::make_shared(length); - if (type_name == "datetime" || type_name == "timestamp") - { - if (!type_support.isSet(MySQLDataTypesSupport::DATETIME64)) - return std::make_shared(); + DataTypePtr res; - if (type_name == "timestamp" && scale == 0) - { - return std::make_shared(); - } - else if (type_name == "datetime" || type_name == "timestamp") - { - return std::make_shared(scale); - } - } - - if (type_support.isSet(MySQLDataTypesSupport::DECIMAL) && (type_name == "numeric" || type_name == "decimal")) + if (type_name == "tinyint") + { + if (is_unsigned) + res = std::make_shared(); + else + res = std::make_shared(); + } + else if (type_name == "smallint") + { + if (is_unsigned) + res = std::make_shared(); + else + res = std::make_shared(); + } + else if (type_name == "int" || type_name == "mediumint") + { + if (is_unsigned) + res = std::make_shared(); + else + res = std::make_shared(); + } + else if (type_name == "bigint") + { + if (is_unsigned) + res = std::make_shared(); + else + res = std::make_shared(); + } + else if (type_name == "float") + res = std::make_shared(); + else if (type_name == "double") + res = std::make_shared(); + else if (type_name == "date") + res = std::make_shared(); + else if (type_name == "binary") + res = std::make_shared(length); + else if (type_name == "datetime" || type_name == "timestamp") + { + if (!type_support.isSet(MySQLDataTypesSupport::DATETIME64)) { - if (precision <= DecimalUtils::maxPrecision()) - return std::make_shared>(precision, scale); - else if (precision <= DecimalUtils::maxPrecision()) - return std::make_shared>(precision, scale); - else if (precision <= DecimalUtils::maxPrecision()) - return std::make_shared>(precision, scale); + res = std::make_shared(); } - + else if (type_name == "timestamp" && scale == 0) + { + res = std::make_shared(); + } + else if (type_name == "datetime" || type_name == "timestamp") + { + res = std::make_shared(scale); + } + } + else if (type_support.isSet(MySQLDataTypesSupport::DECIMAL) && (type_name == "numeric" || type_name == "decimal")) + { + if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + else if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + else if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + } + else + { /// Also String is fallback for all unknown types. - return std::make_shared(); - }(); + res = std::make_shared(); + } if (is_nullable) res = std::make_shared(res); diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index fbd19ce1cd9..1c7a5e24282 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -4,7 +4,6 @@ #include #include "DiskS3.h" #include "Disks/DiskCacheWrapper.h" -#include "Disks/DiskCacheWrapper.cpp" #include "Disks/DiskFactory.h" #include "ProxyConfiguration.h" #include "ProxyListConfiguration.h" diff --git a/src/Functions/appendTrailingCharIfAbsent.cpp b/src/Functions/appendTrailingCharIfAbsent.cpp index 67a3cbabe6d..eb625374707 100644 --- a/src/Functions/appendTrailingCharIfAbsent.cpp +++ b/src/Functions/appendTrailingCharIfAbsent.cpp @@ -92,7 +92,7 @@ private: src_offset = src_offsets[i]; dst_offset += src_length; - if (src_length > 1 && dst_data[dst_offset - 2] != trailing_char_str.front()) + if (src_length > 1 && dst_data[dst_offset - 2] != UInt8(trailing_char_str.front())) { dst_data[dst_offset - 1] = trailing_char_str.front(); dst_data[dst_offset] = 0; diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index e068f3581bd..6d601b1e9ed 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -101,7 +100,7 @@ void PocoHTTPClient::MakeRequestInternal( EnumSize, }; - auto selectMetric = [&request](S3MetricType type) + auto select_metric = [&request](S3MetricType type) { const ProfileEvents::Event events_map[][2] = { {ProfileEvents::S3ReadMicroseconds, ProfileEvents::S3WriteMicroseconds}, @@ -128,12 +127,12 @@ void PocoHTTPClient::MakeRequestInternal( throw Exception("Unsupported request method", ErrorCodes::NOT_IMPLEMENTED); }; - ProfileEvents::increment(selectMetric(S3MetricType::Count)); + ProfileEvents::increment(select_metric(S3MetricType::Count)); - const int MAX_REDIRECT_ATTEMPTS = 10; + static constexpr int max_redirect_attempts = 10; try { - for (int attempt = 0; attempt < MAX_REDIRECT_ATTEMPTS; ++attempt) + for (int attempt = 0; attempt < max_redirect_attempts; ++attempt) { Poco::URI poco_uri(uri); @@ -202,7 +201,7 @@ void PocoHTTPClient::MakeRequestInternal( auto & response_body_stream = session->receiveResponse(poco_response); watch.stop(); - ProfileEvents::increment(selectMetric(S3MetricType::Microseconds), watch.elapsedMicroseconds()); + ProfileEvents::increment(select_metric(S3MetricType::Microseconds), watch.elapsedMicroseconds()); int status_code = static_cast(poco_response.getStatus()); LOG_DEBUG(log, "Response status: {}, {}", status_code, poco_response.getReason()); @@ -214,7 +213,7 @@ void PocoHTTPClient::MakeRequestInternal( uri = location; LOG_DEBUG(log, "Redirecting request to new location: {}", location); - ProfileEvents::increment(selectMetric(S3MetricType::Redirects)); + ProfileEvents::increment(select_metric(S3MetricType::Redirects)); continue; } @@ -240,11 +239,11 @@ void PocoHTTPClient::MakeRequestInternal( if (status_code == 429 || status_code == 503) { // API throttling - ProfileEvents::increment(selectMetric(S3MetricType::Throttling)); + ProfileEvents::increment(select_metric(S3MetricType::Throttling)); } else { - ProfileEvents::increment(selectMetric(S3MetricType::Errors)); + ProfileEvents::increment(select_metric(S3MetricType::Errors)); } } else @@ -261,7 +260,7 @@ void PocoHTTPClient::MakeRequestInternal( response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION); response->SetClientErrorMessage(getCurrentExceptionMessage(false)); - ProfileEvents::increment(selectMetric(S3MetricType::Errors)); + ProfileEvents::increment(select_metric(S3MetricType::Errors)); } } } diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index ba9a61ab922..db7aaf1549b 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -12,9 +12,7 @@ # include # include # include -# include # include -# include # include # include # include diff --git a/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity.cpp b/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity.cpp index d9ddb8e9722..4e7bff0ef41 100644 --- a/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity.cpp +++ b/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity.cpp @@ -3,7 +3,7 @@ #include // I know that inclusion of .cpp is not good at all -#include +#include // NOLINT using namespace DB; static Block getBlockWithSize(size_t required_size_in_bytes, size_t size_of_row_in_bytes) diff --git a/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity_compact_parts.cpp b/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity_compact_parts.cpp index f87293dcd5d..09b24c7dad6 100644 --- a/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity_compact_parts.cpp +++ b/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity_compact_parts.cpp @@ -3,7 +3,7 @@ #include // I know that inclusion of .cpp is not good at all -#include +#include // NOLINT using namespace DB; diff --git a/utils/db-generator/query_db_generator.cpp b/utils/db-generator/query_db_generator.cpp index 88f46325c72..16aa8aa7769 100644 --- a/utils/db-generator/query_db_generator.cpp +++ b/utils/db-generator/query_db_generator.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -16,6 +15,7 @@ #include #include + using ColumnType = uint32_t; using TableAndColumn = std::pair; pcg64 rng; @@ -92,7 +92,7 @@ TableAndColumn get_table_a_column(const std::string & c) } -enum type : ColumnType +enum Type : ColumnType { i = 1, // int @@ -114,9 +114,11 @@ enum type : ColumnType }; -std::map type_definition = { - {type::i, "Int64"}, {type::f, "Float64"}, {type::s, "String"}, {type::d, "Date"}, {type::dt, "DateTime"}, {type::b, "UInt8"} +std::map type_definition = +{ + {Type::i, "Int64"}, {Type::f, "Float64"}, {Type::s, "String"}, {Type::d, "Date"}, {Type::dt, "DateTime"}, {Type::b, "UInt8"} }; + ColumnType time_type(std::string value) { if (value.length() == 12) @@ -124,14 +126,14 @@ ColumnType time_type(std::string value) for (size_t i : {5, 8}) { if (value[i] != '-') - return type::s; + return Type::s; } for (size_t i : {1, 2, 3, 4, 6, 7, 9, 10}) { if (!isdigit(value[i])) - return type::s; + return Type::s; } - return type::d; + return Type::d; } if (value.length() == 21) @@ -139,18 +141,18 @@ ColumnType time_type(std::string value) for (size_t i : {5, 8}) { if (value[i] != '-') - return type::s; + return Type::s; } for (size_t i : {14, 17}) { if (value[i] != '-') - return type::s; + return Type::s; } if (value[11] != '-') - return type::s; - return type::dt; + return Type::s; + return Type::dt; } - return type::s; + return Type::s; } // Casting inner clickhouse parser type to our type ColumnType type_cast(int t) @@ -164,21 +166,21 @@ ColumnType type_cast(int t) case 19: case 20: case 21: - return type::i; + return Type::i; case 3: - return type::f; + return Type::f; case 16: - return type::s; + return Type::s; case 17: - return type::a | type::all; + return Type::a | Type::all; case 18: - return type::t | type::all; + return Type::t | Type::all; } - return type::all; + return Type::all; } @@ -197,28 +199,28 @@ public: , is_array(is_a) {} std::string value{}; - ColumnType type = type::all; + ColumnType type = Type::all; bool is_array = false; }; std::map func_to_return_type = { - {"divide", FuncRet(type::f, "")}, {"e", FuncRet(type::f, "e()")}, {"pi", FuncRet(type::f, "pi()")}, {"exp", FuncRet(type::f, "")}, - {"log", FuncRet(type::f,"")}, {"exp2", FuncRet(type::f, "")}, {"log2", FuncRet(type::f, "")}, {"exp10", FuncRet(type::f, "")}, - {"log10", FuncRet(type::f, "")}, {"sqrt", FuncRet(type::f, "")}, {"cbrt", FuncRet(type::f, "")}, {"erf", FuncRet(type::f, "")}, - {"erfc", FuncRet(type::f, "")}, {"lgamma", FuncRet(type::f, "")}, {"tgamma", FuncRet(type::f, "")}, {"sin", FuncRet(type::f, "")}, - {"cos", FuncRet(type::f, "")}, {"tan", FuncRet(type::f, "")}, {"asin", FuncRet(type::f, "")}, {"acos", FuncRet(type::f, "")}, - {"atan", FuncRet(type::f, "")}, {"pow", FuncRet(type::f, "")}, {"splitbystring", FuncRet(type::s | type::a,"")}, - {"splitbychar", FuncRet(type::s | type::a, "")}, {"alphatokens", FuncRet(type::s | type::a, "")}, {"toyear", FuncRet(type::i, "")}, - {"tomonth", FuncRet(type::i, "")}, {"todayofmonth", FuncRet(type::i, "")}, {"tohour", FuncRet(type::dt, "")}, {"tominute", FuncRet(type::dt, "")}, - {"toseconds", FuncRet(type::dt, "")}, {"tounixtimestamp", FuncRet(type::i, "")}, {"tostartofyear", FuncRet(type::dt | type::d, "")}, - {"tostartofquater",FuncRet(type::dt | type::d, "")}, {"tostartofmonth", FuncRet(type::dt | type::d, "")}, {"tomonday", FuncRet(type::dt | type::d, "")}, - {"tostartoffiveminutes", FuncRet(type::dt, "")}, {"tostartoftenminutes", FuncRet(type::dt, "")}, {"tostartoffifteenminutes", FuncRet(type::dt, "")}, - {"tostartofinterval", FuncRet(type::dt, "")}, {"totime", FuncRet(type::dt, "")}, {"torelativemonthnum", FuncRet(type::i, "")}, - {"torelativeweeknum", FuncRet(type::i, "")}, {"torelativedaynum", FuncRet(type::i, "")}, {"torelativehournum", FuncRet(type::i, "")}, - {"torelativeminutenum", FuncRet(type::i, "")}, {"torelativesecondsnum", FuncRet(type::i, "")}, {"datediff", FuncRet(type::d | type::dt, "")}, - {"formatdatetime", FuncRet(type::s, "")}, {"now", FuncRet(type::dt | type::d, "now()")}, {"today", FuncRet(type::d | type::dt, "today()")}, - {"yesterday", FuncRet(type::d | type::dt, "yesterday()")} + {"divide", FuncRet(Type::f, "")}, {"e", FuncRet(Type::f, "e()")}, {"pi", FuncRet(Type::f, "pi()")}, {"exp", FuncRet(Type::f, "")}, + {"log", FuncRet(Type::f,"")}, {"exp2", FuncRet(Type::f, "")}, {"log2", FuncRet(Type::f, "")}, {"exp10", FuncRet(Type::f, "")}, + {"log10", FuncRet(Type::f, "")}, {"sqrt", FuncRet(Type::f, "")}, {"cbrt", FuncRet(Type::f, "")}, {"erf", FuncRet(Type::f, "")}, + {"erfc", FuncRet(Type::f, "")}, {"lgamma", FuncRet(Type::f, "")}, {"tgamma", FuncRet(Type::f, "")}, {"sin", FuncRet(Type::f, "")}, + {"cos", FuncRet(Type::f, "")}, {"tan", FuncRet(Type::f, "")}, {"asin", FuncRet(Type::f, "")}, {"acos", FuncRet(Type::f, "")}, + {"atan", FuncRet(Type::f, "")}, {"pow", FuncRet(Type::f, "")}, {"splitbystring", FuncRet(Type::s | Type::a,"")}, + {"splitbychar", FuncRet(Type::s | Type::a, "")}, {"alphatokens", FuncRet(Type::s | Type::a, "")}, {"toyear", FuncRet(Type::i, "")}, + {"tomonth", FuncRet(Type::i, "")}, {"todayofmonth", FuncRet(Type::i, "")}, {"tohour", FuncRet(Type::dt, "")}, {"tominute", FuncRet(Type::dt, "")}, + {"toseconds", FuncRet(Type::dt, "")}, {"tounixtimestamp", FuncRet(Type::i, "")}, {"tostartofyear", FuncRet(Type::dt | Type::d, "")}, + {"tostartofquater",FuncRet(Type::dt | Type::d, "")}, {"tostartofmonth", FuncRet(Type::dt | Type::d, "")}, {"tomonday", FuncRet(Type::dt | Type::d, "")}, + {"tostartoffiveminutes", FuncRet(Type::dt, "")}, {"tostartoftenminutes", FuncRet(Type::dt, "")}, {"tostartoffifteenminutes", FuncRet(Type::dt, "")}, + {"tostartofinterval", FuncRet(Type::dt, "")}, {"totime", FuncRet(Type::dt, "")}, {"torelativemonthnum", FuncRet(Type::i, "")}, + {"torelativeweeknum", FuncRet(Type::i, "")}, {"torelativedaynum", FuncRet(Type::i, "")}, {"torelativehournum", FuncRet(Type::i, "")}, + {"torelativeminutenum", FuncRet(Type::i, "")}, {"torelativesecondsnum", FuncRet(Type::i, "")}, {"datediff", FuncRet(Type::d | Type::dt, "")}, + {"formatdatetime", FuncRet(Type::s, "")}, {"now", FuncRet(Type::dt | Type::d, "now()")}, {"today", FuncRet(Type::d | Type::dt, "today()")}, + {"yesterday", FuncRet(Type::d | Type::dt, "yesterday()")} }; std::set func_args_same_types = { @@ -226,23 +228,23 @@ std::set func_args_same_types = { }; std::map func_to_param_type = { - {"tostartofminute", type::dt}, {"plus", type::i | type::f | type::d | type::dt}, {"multiply", type::i | type::f}, - {"minus", type::i | type::f | type::d | type::dt}, {"negate", type::i | type::f}, {"divide", type::i | type::f}, - {"abs", type::i | type::f}, {"gcd", type::i | type::f}, {"lcm", type::i | type::f}, {"bitnot", type::i}, {"bitshiftleft", type::i}, - {"bitshiftright", type::i}, {"bittest", type::i}, {"exp", type::i | type::f}, {"log", type::i | type::f}, - {"exp2", type::i | type::f}, {"log2", type::i | type::f}, {"exp10", type::i | type::f}, {"log10", type::i | type::f}, - {"sqrt", type::i | type::f}, {"cbrt", type::i | type::f}, {"erf", type::i | type::f}, {"erfc", type::i | type::f}, - {"lgamma", type::i | type::f}, {"tgamma", type::i | type::f}, {"sin", type::i | type::f}, {"cos", type::i | type::f}, - {"tan", type::i | type::f}, {"asin", type::i | type::f}, {"acos", type::i | type::f}, {"atan", type::i | type::f}, - {"pow", type::i | type::f}, {"arrayjoin", type::all | type::a}, {"substring", type::s}, {"splitbystring", type::s}, {"splitbychar", type::s}, - {"alphatokens", type::s}, {"toyear", type::d | type::dt}, {"tomonth", type::d | type::dt}, {"todayofmonth", type::d | type::dt}, {"tohour", type::dt}, - {"tominute", type::dt}, {"tosecond", type::dt}, {"touixtimestamp", type::dt}, {"tostartofyear", type::d | type::dt}, - {"tostartofquarter", type::d | type::dt}, {"tostartofmonth", type::d | type::dt}, {"tomonday", type::d | type::dt}, - {"tostartoffiveminute", type::dt}, {"tostartoftenminutes", type::dt}, {"tostartoffifteenminutes", type::d | type::dt}, - {"tostartofinterval", type::d | type::dt}, {"totime", type::d | type::dt}, {"torelativehonthnum", type::d | type::dt}, - {"torelativeweeknum", type::d | type::dt}, {"torelativedaynum", type::d | type::dt}, {"torelativehournum", type::d | type::dt}, - {"torelativeminutenum", type::d | type::dt}, {"torelativesecondnum", type::d | type::dt}, {"datediff", type::d | type::dt}, - {"formatdatetime", type::dt} + {"tostartofminute", Type::dt}, {"plus", Type::i | Type::f | Type::d | Type::dt}, {"multiply", Type::i | Type::f}, + {"minus", Type::i | Type::f | Type::d | Type::dt}, {"negate", Type::i | Type::f}, {"divide", Type::i | Type::f}, + {"abs", Type::i | Type::f}, {"gcd", Type::i | Type::f}, {"lcm", Type::i | Type::f}, {"bitnot", Type::i}, {"bitshiftleft", Type::i}, + {"bitshiftright", Type::i}, {"bittest", Type::i}, {"exp", Type::i | Type::f}, {"log", Type::i | Type::f}, + {"exp2", Type::i | Type::f}, {"log2", Type::i | Type::f}, {"exp10", Type::i | Type::f}, {"log10", Type::i | Type::f}, + {"sqrt", Type::i | Type::f}, {"cbrt", Type::i | Type::f}, {"erf", Type::i | Type::f}, {"erfc", Type::i | Type::f}, + {"lgamma", Type::i | Type::f}, {"tgamma", Type::i | Type::f}, {"sin", Type::i | Type::f}, {"cos", Type::i | Type::f}, + {"tan", Type::i | Type::f}, {"asin", Type::i | Type::f}, {"acos", Type::i | Type::f}, {"atan", Type::i | Type::f}, + {"pow", Type::i | Type::f}, {"arrayjoin", Type::all | Type::a}, {"substring", Type::s}, {"splitbystring", Type::s}, {"splitbychar", Type::s}, + {"alphatokens", Type::s}, {"toyear", Type::d | Type::dt}, {"tomonth", Type::d | Type::dt}, {"todayofmonth", Type::d | Type::dt}, {"tohour", Type::dt}, + {"tominute", Type::dt}, {"tosecond", Type::dt}, {"touixtimestamp", Type::dt}, {"tostartofyear", Type::d | Type::dt}, + {"tostartofquarter", Type::d | Type::dt}, {"tostartofmonth", Type::d | Type::dt}, {"tomonday", Type::d | Type::dt}, + {"tostartoffiveminute", Type::dt}, {"tostartoftenminutes", Type::dt}, {"tostartoffifteenminutes", Type::d | Type::dt}, + {"tostartofinterval", Type::d | Type::dt}, {"totime", Type::d | Type::dt}, {"torelativehonthnum", Type::d | Type::dt}, + {"torelativeweeknum", Type::d | Type::dt}, {"torelativedaynum", Type::d | Type::dt}, {"torelativehournum", Type::d | Type::dt}, + {"torelativeminutenum", Type::d | Type::dt}, {"torelativesecondnum", Type::d | Type::dt}, {"datediff", Type::d | Type::dt}, + {"formatdatetime", Type::dt} }; @@ -252,7 +254,7 @@ public: TableAndColumn name; std::set equals; std::set values; - ColumnType type = type::all; + ColumnType type = Type::all; bool is_array = false; Column() = default; @@ -260,7 +262,7 @@ public: explicit Column(const std::string & column_name) { name = std::make_pair("", column_name); - type = type::all; + type = Type::all; } void merge(Column other) @@ -275,15 +277,15 @@ public: void printType() const { - if (type & type::i) + if (type & Type::i) std::cout << "I"; - if (type & type::f) + if (type & Type::f) std::cout << "F"; - if (type & type::s) + if (type & Type::s) std::cout << "S"; - if (type & type::d) + if (type & Type::d) std::cout << "D"; - if (type & type::dt) + if (type & Type::dt) std::cout << "DT"; if (is_array) std::cout << "ARR"; @@ -307,22 +309,22 @@ public: std::string generateOneValue() const { - if (type & type::i) + if (type & Type::i) return randomInteger(); - if (type & type::f) + if (type & Type::f) return randomFloat(); - if (type & type::d) + if (type & Type::d) return randomDate(); - if (type & type::dt) + if (type & Type::dt) return randomDatetime(); - if (type & type::s) + if (type & Type::s) return "'" + randomString(rng() % 40) + "'"; - if (type & type::b) + if (type & Type::b) return "0"; return ""; @@ -332,7 +334,7 @@ public: { if (values.size() > 2 && amount == 0) return false; - while (values.size() < 1 or amount > 0) + while (values.empty() or amount > 0) { amount -= 1; if (is_array) @@ -357,18 +359,18 @@ public: void unifyType() { - if (type & type::i) - type = type::i; - else if (type & type::f) - type = type::f; - else if (type & type::d) - type = type::d; - else if (type & type::dt) - type = type::dt; - else if (type & type::s) - type = type::s; - else if (type & type::b) - type = type::b; + if (type & Type::i) + type = Type::i; + else if (type & Type::f) + type = Type::f; + else if (type & Type::d) + type = Type::d; + else if (type & Type::dt) + type = Type::dt; + else if (type & Type::s) + type = Type::s; + else if (type & Type::b) + type = Type::b; else throw std::runtime_error("Error in determination column type " + name.first + '.' + name.second); } @@ -381,13 +383,15 @@ decartMul( std::set & mul) { std::set> result; - for (auto v : prev) - for (auto m : mul) + for (const auto & v : prev) + { + for (const auto & m : mul) { std::vector tmp = v; tmp.push_back(m); result.insert(tmp); } + } return result; } @@ -438,7 +442,7 @@ public: { name = other.name; columns.insert(other.columns.begin(), other.columns.end()); - for (auto desc : other.column_description) + for (const auto & desc : other.column_description) column_description[desc.first].merge(desc.second); } @@ -576,7 +580,7 @@ public: void merge(TableList other) { - for (auto table : other.tables) + for (const auto & table : other.tables) tables[table.first].merge(table.second); nested.insert(other.nested.begin(), other.nested.end()); if (main_table.empty()) @@ -617,14 +621,14 @@ FuncRet arrayJoinFunc(DB::ASTPtr ch, std::map & columns) for (const auto & indent : indents) { auto c = Column(indent); - c.type = type::all; + c.type = Type::all; c.is_array = true; if (columns.count(indent)) columns[indent].merge(c); else columns[indent] = c; } - FuncRet r(type::all, ""); + FuncRet r(Type::all, ""); return r; } return FuncRet(); @@ -637,7 +641,7 @@ FuncRet inFunc(DB::ASTPtr ch, std::map & columns) { std::set indents{}; std::set values{}; - ColumnType type_value = type::all; + ColumnType type_value = Type::all; for (auto & arg : x->arguments->children) { @@ -656,20 +660,20 @@ FuncRet inFunc(DB::ASTPtr ch, std::map & columns) for (auto & val : arr_values) { type = type_cast(val.getType()); - if (type == type::s || type == type::d || type == type::dt) + if (type == Type::s || type == Type::d || type == Type::dt) type = time_type(applyVisitor(DB::FieldVisitorToString(), val)); type_value &= type; values.insert(applyVisitor(DB::FieldVisitorToString(), val)); } }; - if (type & type::a) + if (type & Type::a) { auto arr_values = literal->value.get(); routine(arr_values); } - if (type & type::a) + if (type & Type::a) { auto arr_values = literal->value.get(); routine(arr_values); @@ -685,7 +689,7 @@ FuncRet inFunc(DB::ASTPtr ch, std::map & columns) else f = handlers[""]; FuncRet ret = f(arg, columns); - if (ret.value != "") + if (!ret.value.empty()) { values.insert(ret.value); } @@ -703,7 +707,7 @@ FuncRet inFunc(DB::ASTPtr ch, std::map & columns) else columns[indent] = c; } - FuncRet r(type::b | type::i, ""); + FuncRet r(Type::b | Type::i, ""); return r; } return FuncRet(); @@ -716,7 +720,7 @@ FuncRet arrayFunc(DB::ASTPtr ch, std::map & columns) { std::set indents = {}; std::string value = "["; - ColumnType type_value = type::i | type::f | type::d | type::dt | type::s; + ColumnType type_value = Type::i | Type::f | Type::d | Type::dt | Type::s; bool no_indent = true; for (const auto & arg : x->arguments->children) { @@ -730,7 +734,7 @@ FuncRet arrayFunc(DB::ASTPtr ch, std::map & columns) if (literal) { ColumnType type = type_cast(literal->value.getType()); - if (type == type::s || type == type::d || type == type::dt) + if (type == Type::s || type == Type::d || type == Type::dt) type = time_type(value); type_value &= type; @@ -764,7 +768,7 @@ FuncRet arithmeticFunc(DB::ASTPtr ch, std::map & columns) { std::set indents = {}; std::set values = {}; - ColumnType type_value = type::i | type::f | type::d | type::dt; + ColumnType type_value = Type::i | Type::f | Type::d | Type::dt; ColumnType args_types = 0; bool no_indent = true; for (auto & arg : x->arguments->children) @@ -793,11 +797,11 @@ FuncRet arithmeticFunc(DB::ASTPtr ch, std::map & columns) } args_types |= type; } - if (args_types & (type::d | type::dt)) - type_value -= type::f; - if (args_types & type::f) - type_value -= type::d | type::dt; - for (auto indent : indents) + if (args_types & (Type::d | Type::dt)) + type_value -= Type::f; + if (args_types & Type::f) + type_value -= Type::d | Type::dt; + for (const auto & indent : indents) { auto c = Column(indent); c.type = type_value; @@ -807,14 +811,14 @@ FuncRet arithmeticFunc(DB::ASTPtr ch, std::map & columns) columns[indent] = c; } ColumnType ret_type = 0; - if (args_types & type::dt) - ret_type = type::dt; - else if (args_types & type::d) - ret_type = type::d | type::dt; - else if (args_types & type::f) - ret_type = type::f; + if (args_types & Type::dt) + ret_type = Type::dt; + else if (args_types & Type::d) + ret_type = Type::d | Type::dt; + else if (args_types & Type::f) + ret_type = Type::f; else - ret_type = type::d | type::f | type::dt | type::i; + ret_type = Type::d | Type::f | Type::dt | Type::i; FuncRet r(ret_type, ""); if (no_indent) { @@ -833,7 +837,7 @@ FuncRet likeFunc(DB::ASTPtr ch, std::map & columns) { std::set indents = {}; std::set values = {}; - ColumnType type_value = type::s; + ColumnType type_value = Type::s; for (auto & arg : x->arguments->children) { auto ident = std::dynamic_pointer_cast(arg); @@ -866,7 +870,7 @@ FuncRet likeFunc(DB::ASTPtr ch, std::map & columns) else columns[indent] = c; } - FuncRet r(type::b, ""); + FuncRet r(Type::b, ""); return r; } return FuncRet(); @@ -874,22 +878,22 @@ FuncRet likeFunc(DB::ASTPtr ch, std::map & columns) FuncRet simpleFunc(DB::ASTPtr ch, std::map & columns) { - auto X = std::dynamic_pointer_cast(ch); - if (X) + auto x = std::dynamic_pointer_cast(ch); + if (x) { std::set indents = {}; std::set values = {}; - ColumnType type_value = type::all; + ColumnType type_value = Type::all; bool is_array = false; bool no_indent = true; - if (func_to_param_type.count(boost::algorithm::to_lower_copy(X->name))) + if (func_to_param_type.count(boost::algorithm::to_lower_copy(x->name))) { - type_value &= func_to_param_type[boost::algorithm::to_lower_copy(X->name)]; - is_array = func_to_param_type[boost::algorithm::to_lower_copy(X->name)] & type::a; + type_value &= func_to_param_type[boost::algorithm::to_lower_copy(x->name)]; + is_array = func_to_param_type[boost::algorithm::to_lower_copy(x->name)] & Type::a; } - for (auto arg : X->arguments->children) + for (const auto & arg : x->arguments->children) { - ColumnType type = type::all; + ColumnType type = Type::all; std::string value; auto ident = std::dynamic_pointer_cast(arg); if (ident) @@ -902,7 +906,7 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map & columns) { value = applyVisitor(DB::FieldVisitorToString(), literal->value); type = type_cast(literal->value.getType()); - is_array |= type & type::a; + is_array |= type & Type::a; } auto subfunc = std::dynamic_pointer_cast(arg); if (subfunc) @@ -922,31 +926,31 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map & columns) } if (!value.empty()) { - if (type == type::i) + if (type == Type::i) { values.insert(value); values.insert(value + " + " + randomInteger(1, 10)); values.insert(value + " - " + randomInteger(1, 10)); } - if (type == type::f) + if (type == Type::f) { values.insert(value); values.insert(value + " + " + randomFloat(1, 10)); values.insert(value + " - " + randomFloat(1, 10)); } - if (type & type::s || type & type::d || type & type::dt) + if (type & Type::s || type & Type::d || type & Type::dt) { - if (type == type::s) + if (type == Type::s) type = time_type(value); - if (type == type::s) + if (type == Type::s) values.insert(value); - if (type & type::d) + if (type & Type::d) { values.insert(value); values.insert("toDate(" + value + ") + " + randomInteger(1, 10)); values.insert("toDate(" + value + ") - " + randomInteger(1, 10)); } - else if (type & type::dt) + else if (type & Type::dt) { values.insert(value); values.insert( @@ -956,7 +960,7 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map & columns) } } } - if (func_args_same_types.count(boost::algorithm::to_lower_copy(X->name))) + if (func_args_same_types.count(boost::algorithm::to_lower_copy(x->name))) type_value &= type; } for (const auto & indent : indents) @@ -965,7 +969,7 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map & columns) c.type = type_value; c.is_array = is_array; if (func_args_same_types.count( - boost::algorithm::to_lower_copy(X->name))) + boost::algorithm::to_lower_copy(x->name))) c.values = values; for (const auto & ind : indents) if (ind != indent) @@ -976,31 +980,31 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map & columns) else columns[indent] = c; } - if (func_to_return_type.count(boost::algorithm::to_lower_copy(X->name))) + if (func_to_return_type.count(boost::algorithm::to_lower_copy(x->name))) { if (no_indent) { std::ostringstream ss; formatAST(*ch, ss); - auto r = func_to_return_type[boost::algorithm::to_lower_copy(X->name)]; + auto r = func_to_return_type[boost::algorithm::to_lower_copy(x->name)]; r.value = ss.str(); return r; } - return func_to_return_type[boost::algorithm::to_lower_copy(X->name)]; + return func_to_return_type[boost::algorithm::to_lower_copy(x->name)]; } else if (func_to_param_type.count( - boost::algorithm::to_lower_copy(X->name))) + boost::algorithm::to_lower_copy(x->name))) { if (no_indent) { std::ostringstream ss; formatAST(*ch, ss); return FuncRet( - func_to_param_type[boost::algorithm::to_lower_copy(X->name)], + func_to_param_type[boost::algorithm::to_lower_copy(x->name)], ss.str()); } return FuncRet( - func_to_param_type[boost::algorithm::to_lower_copy(X->name)], + func_to_param_type[boost::algorithm::to_lower_copy(x->name)], ""); } } @@ -1070,7 +1074,7 @@ connectedEqualityFind( std::set & visited) { std::set result; - for (auto & column : now.equals) + for (const auto & column : now.equals) if (!visited.count(column)) { visited.insert(column); @@ -1117,18 +1121,18 @@ unificateColumns( result[column.second.name.first + "." + column.second.name.second], result, visited); - for (auto c : equal) + for (const auto & c : equal) result[c.first + "." + c.second].equals = equal; } for (auto & column : result) - for (auto e : column.second.equals) + for (const auto & e : column.second.equals) column.second.merge(result[e.first + "." + e.second]); for (auto & column : result) { column.second.unifyType(); if (column.second.generateValues()) - for (auto e : column.second.equals) + for (const auto & e : column.second.equals) result[e.first + "." + e.second].merge(column.second); } @@ -1174,18 +1178,18 @@ void parseSelectQuery(DB::ASTPtr ast, TableList & all_tables) for (auto & child : x->children) { auto ch = std::dynamic_pointer_cast(child); - auto TEast = std::dynamic_pointer_cast(ch->table_expression); - if (TEast && TEast->database_and_table_name) + auto table_expression_ast = std::dynamic_pointer_cast(ch->table_expression); + if (table_expression_ast && table_expression_ast->database_and_table_name) { - auto table_name = *(getIndent(TEast->database_and_table_name).begin()); + auto table_name = *(getIndent(table_expression_ast->database_and_table_name).begin()); all_tables.addTable(table_name); auto alias = getAlias(ch); if (!alias.empty()) all_tables.aliases[alias] = table_name; } - if (TEast && TEast->subquery) + if (table_expression_ast && table_expression_ast->subquery) { - for (auto select : getSelect(TEast->subquery)) + for (const auto & select : getSelect(table_expression_ast->subquery)) { TableList local; parseSelectQuery(select, local); From 1238822785124e7a37a9cfd73279ab471bd4ffbb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 13:37:40 +0300 Subject: [PATCH 205/273] Fix clang-tidy --- src/IO/S3/PocoHTTPClient.cpp | 2 +- src/IO/S3/PocoHTTPClient.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 6d601b1e9ed..0604c5f1360 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -78,7 +78,7 @@ std::shared_ptr PocoHTTPClient::MakeRequest( return response; } -void PocoHTTPClient::MakeRequestInternal( +void PocoHTTPClient::makeRequestInternal( Aws::Http::HttpRequest & request, std::shared_ptr & response, Aws::Utils::RateLimits::RateLimiterInterface *, diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index a94a08e217d..eefc85fae70 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -37,7 +37,7 @@ public: Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const override; private: - void MakeRequestInternal( + void makeRequestInternal( Aws::Http::HttpRequest & request, std::shared_ptr & response, Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, From eb260f6a89d95646487f6421a8c2ca5829a86933 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 13:43:26 +0300 Subject: [PATCH 206/273] Fix bad code --- src/IO/S3/PocoHTTPClient.cpp | 7 +++++++ src/IO/S3/PocoHTTPClientFactory.cpp | 6 ++++++ src/IO/S3/PocoHTTPResponseStream.cpp | 7 +++++++ 3 files changed, 20 insertions(+) diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 0604c5f1360..3735851931a 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -1,3 +1,7 @@ +#include + +#if USE_AWS_S3 + #include "PocoHTTPClient.h" #include @@ -14,6 +18,7 @@ #include #include + namespace ProfileEvents { extern const Event S3ReadMicroseconds; @@ -264,3 +269,5 @@ void PocoHTTPClient::makeRequestInternal( } } } + +#endif diff --git a/src/IO/S3/PocoHTTPClientFactory.cpp b/src/IO/S3/PocoHTTPClientFactory.cpp index 68f6a6b9823..b257f96e383 100644 --- a/src/IO/S3/PocoHTTPClientFactory.cpp +++ b/src/IO/S3/PocoHTTPClientFactory.cpp @@ -1,3 +1,7 @@ +#include + +#if USE_AWS_S3 + #include "PocoHTTPClientFactory.h" #include @@ -32,3 +36,5 @@ std::shared_ptr PocoHTTPClientFactory::CreateHttpRequest } } + +#endif diff --git a/src/IO/S3/PocoHTTPResponseStream.cpp b/src/IO/S3/PocoHTTPResponseStream.cpp index 0a198268f2e..b35188d9498 100644 --- a/src/IO/S3/PocoHTTPResponseStream.cpp +++ b/src/IO/S3/PocoHTTPResponseStream.cpp @@ -1,3 +1,8 @@ +#include + +#if USE_AWS_S3 + + #include "PocoHTTPResponseStream.h" #include @@ -10,3 +15,5 @@ PocoHTTPResponseStream::PocoHTTPResponseStream(std::shared_ptr Date: Sat, 19 Sep 2020 15:24:43 +0300 Subject: [PATCH 207/273] Fix clang-tidy --- src/IO/S3/PocoHTTPClient.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 3735851931a..c34d7719131 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -69,7 +69,7 @@ std::shared_ptr PocoHTTPClient::MakeRequest( Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const { auto response = Aws::MakeShared("PocoHTTPClient", request); - MakeRequestInternal(request, response, readLimiter, writeLimiter); + makeRequestInternal(request, response, readLimiter, writeLimiter); return response; } @@ -79,7 +79,7 @@ std::shared_ptr PocoHTTPClient::MakeRequest( Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const { auto response = Aws::MakeShared("PocoHTTPClient", request); - MakeRequestInternal(*request, response, readLimiter, writeLimiter); + makeRequestInternal(*request, response, readLimiter, writeLimiter); return response; } From 46aa307d0ab958b3057064e1fbc6a1792a4c575a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 19 Sep 2020 17:19:06 +0300 Subject: [PATCH 208/273] fix EINVAL in renameat2 on MacOS --- src/Common/renameat2.cpp | 26 ++++++++++++++++++-------- src/Common/renameat2.h | 3 +++ src/Databases/DatabaseAtomic.cpp | 7 +++---- 3 files changed, 24 insertions(+), 12 deletions(-) diff --git a/src/Common/renameat2.cpp b/src/Common/renameat2.cpp index 323b72267a6..5139a165c91 100644 --- a/src/Common/renameat2.cpp +++ b/src/Common/renameat2.cpp @@ -48,8 +48,10 @@ static bool supportsRenameat2Impl() #if defined(__NR_renameat2) -static void renameat2(const std::string & old_path, const std::string & new_path, int flags) +static bool renameat2(const std::string & old_path, const std::string & new_path, int flags) { + if(!supportsRenameat2()) + return false; if (old_path.empty() || new_path.empty()) throw Exception("Cannot rename " + old_path + " to " + new_path + ": path is empty", ErrorCodes::LOGICAL_ERROR); @@ -57,7 +59,14 @@ static void renameat2(const std::string & old_path, const std::string & new_path /// int newdirfd (ignored for absolute newpath), const char *newpath, /// unsigned int flags if (0 == syscall(__NR_renameat2, AT_FDCWD, old_path.c_str(), AT_FDCWD, new_path.c_str(), flags)) - return; + return true; + + /// EINVAL means that filesystem does not support one of the flags. + /// It also may happen when running clickhouse in docker with Mac OS as a host OS. + /// supportsRenameat2() with uname is not enough in this case, because virtualized Linux kernel is used. + /// Other cases when EINVAL can be returned should never happen. + if (errno == EINVAL) + return false; if (errno == EEXIST) throwFromErrno("Cannot rename " + old_path + " to " + new_path + " because the second path already exists", ErrorCodes::ATOMIC_RENAME_FAIL); @@ -104,18 +113,19 @@ bool supportsRenameat2() void renameNoReplace(const std::string & old_path, const std::string & new_path) { - if (supportsRenameat2()) - renameat2(old_path, new_path, RENAME_NOREPLACE); - else + if (!renameat2(old_path, new_path, RENAME_NOREPLACE)) renameNoReplaceFallback(old_path, new_path); } void renameExchange(const std::string & old_path, const std::string & new_path) { - if (supportsRenameat2()) - renameat2(old_path, new_path, RENAME_EXCHANGE); - else + if (!renameat2(old_path, new_path, RENAME_EXCHANGE)) renameExchangeFallback(old_path, new_path); } +bool renameExchangeIfSupported(const std::string & old_path, const std::string & new_path) +{ + return renameat2(old_path, new_path, RENAME_EXCHANGE); +} + } diff --git a/src/Common/renameat2.h b/src/Common/renameat2.h index 333f85541f1..141c5d385c5 100644 --- a/src/Common/renameat2.h +++ b/src/Common/renameat2.h @@ -14,4 +14,7 @@ void renameNoReplace(const std::string & old_path, const std::string & new_path) /// Atomically exchange oldpath and newpath. Throw exception if some of them does not exist void renameExchange(const std::string & old_path, const std::string & new_path); +/// Returns false instead of throwing exception if renameat2 is not supported +bool renameExchangeIfSupported(const std::string & old_path, const std::string & new_path); + } diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index e0f72436422..ed17a8eccb1 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -277,7 +277,7 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path) { - bool check_file_exists = supportsRenameat2(); + bool check_file_exists = true; SCOPE_EXIT({ std::error_code code; if (check_file_exists) std::filesystem::remove(table_metadata_tmp_path, code); }); std::unique_lock lock{mutex}; @@ -286,9 +286,8 @@ void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & if (table_id.uuid != actual_table_id.uuid) throw Exception("Cannot alter table because it was renamed", ErrorCodes::CANNOT_ASSIGN_ALTER); - if (check_file_exists) - renameExchange(table_metadata_tmp_path, table_metadata_path); - else + check_file_exists = renameExchangeIfSupported(table_metadata_tmp_path, table_metadata_path); + if (!check_file_exists) std::filesystem::rename(table_metadata_tmp_path, table_metadata_path); } From 059646743bebc4c16ded71d65cb0e58c414b8f0c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 17:20:00 +0300 Subject: [PATCH 209/273] Remove OpenCL --- CMakeLists.txt | 8 - cmake/find/opencl.cmake | 25 -- src/CMakeLists.txt | 5 - src/Columns/ColumnVector.cpp | 33 -- src/Columns/ColumnVector.h | 2 - src/Columns/IColumn.h | 11 - src/Common/BitonicSort.h | 221 ------------ src/Common/ErrorCodes.cpp | 1 - src/Common/oclBasics.h | 354 -------------------- src/Common/tests/CMakeLists.txt | 5 - src/Common/tests/bitonic_sort.cpp | 174 ---------- src/Core/Settings.h | 2 - src/Core/SettingsEnums.cpp | 5 - src/Core/SettingsEnums.h | 9 - src/Core/SortDescription.h | 10 +- src/Interpreters/InterpreterSelectQuery.cpp | 5 +- src/Interpreters/sortBlock.cpp | 7 +- 17 files changed, 7 insertions(+), 870 deletions(-) delete mode 100644 cmake/find/opencl.cmake delete mode 100644 src/Common/BitonicSort.h delete mode 100644 src/Common/oclBasics.h delete mode 100644 src/Common/tests/bitonic_sort.cpp diff --git a/CMakeLists.txt b/CMakeLists.txt index cf97b2c40ff..a879ad9b076 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -404,7 +404,6 @@ include (cmake/find/amqpcpp.cmake) include (cmake/find/capnp.cmake) include (cmake/find/llvm.cmake) include (cmake/find/termcap.cmake) # for external static llvm -include (cmake/find/opencl.cmake) include (cmake/find/h3.cmake) include (cmake/find/libxml2.cmake) include (cmake/find/brotli.cmake) @@ -450,13 +449,6 @@ include (cmake/find/mysqlclient.cmake) # When testing for memory leaks with Valgrind, don't link tcmalloc or jemalloc. -if (USE_OPENCL) - if (OS_DARWIN) - set(OPENCL_LINKER_FLAGS "-framework OpenCL") - set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${OPENCL_LINKER_FLAGS}") - endif () -endif () - include (cmake/print_flags.cmake) if (TARGET global-group) diff --git a/cmake/find/opencl.cmake b/cmake/find/opencl.cmake deleted file mode 100644 index 2b0cc7c5dd4..00000000000 --- a/cmake/find/opencl.cmake +++ /dev/null @@ -1,25 +0,0 @@ -# TODO: enable by default -if(0) - option(ENABLE_OPENCL "Enable OpenCL support" ${ENABLE_LIBRARIES}) -endif() - -if(NOT ENABLE_OPENCL) - return() -endif() - -# Intel OpenCl driver: sudo apt install intel-opencl-icd -# @sa https://github.com/intel/compute-runtime/releases - -# OpenCL applications should link with ICD loader -# sudo apt install opencl-headers ocl-icd-libopencl1 -# sudo ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so -# TODO: add https://github.com/OCL-dev/ocl-icd as submodule instead - -find_package(OpenCL) -if(OpenCL_FOUND) - set(USE_OPENCL 1) -else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't enable OpenCL support") -endif() - -message(STATUS "Using opencl=${USE_OPENCL}: ${OpenCL_INCLUDE_DIRS} : ${OpenCL_LIBRARIES}") diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index b6e8c395b26..b058ab749b6 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -378,11 +378,6 @@ if (USE_BROTLI) target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${BROTLI_INCLUDE_DIR}) endif() -if (USE_OPENCL) - target_link_libraries (clickhouse_common_io PRIVATE ${OpenCL_LIBRARIES}) - target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${OpenCL_INCLUDE_DIRS}) -endif () - if (USE_CASSANDRA) dbms_target_link_libraries(PUBLIC ${CASSANDRA_LIBRARY}) dbms_target_include_directories (SYSTEM BEFORE PUBLIC ${CASS_INCLUDE_DIR}) diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 829dc7d7aaa..733a1510f93 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -19,15 +19,6 @@ #include -#if !defined(ARCADIA_BUILD) -# include -# if USE_OPENCL -# include "Common/BitonicSort.h" // Y_IGNORE -# endif -#else -#undef USE_OPENCL -#endif - #ifdef __SSE2__ #include #endif @@ -39,7 +30,6 @@ namespace ErrorCodes { extern const int PARAMETER_OUT_OF_BOUND; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; - extern const int OPENCL_ERROR; extern const int LOGICAL_ERROR; } @@ -147,29 +137,6 @@ namespace }; } -template -void ColumnVector::getSpecialPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, - IColumn::SpecialSort special_sort) const -{ - if (special_sort == IColumn::SpecialSort::OPENCL_BITONIC) - { -#if !defined(ARCADIA_BUILD) -#if USE_OPENCL - if (!limit || limit >= data.size()) - { - res.resize(data.size()); - - if (data.empty() || BitonicSort::getInstance().sort(data, res, !reverse)) - return; - } -#else - throw DB::Exception("'special_sort = bitonic' specified but OpenCL not available", DB::ErrorCodes::OPENCL_ERROR); -#endif -#endif - } - - getPermutation(reverse, limit, nan_direction_hint, res); -} template void ColumnVector::getPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 55ab67d6214..c6600ca7e31 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -218,8 +218,6 @@ public: } void getPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override; - void getSpecialPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, - IColumn::SpecialSort) const override; void updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges& equal_range) const override; diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 40ff0649f4f..14e6a9d7eed 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -267,17 +267,6 @@ public: */ virtual void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const = 0; - enum class SpecialSort - { - NONE = 0, - OPENCL_BITONIC, - }; - - virtual void getSpecialPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, SpecialSort) const - { - getPermutation(reverse, limit, nan_direction_hint, res); - } - /*in updatePermutation we pass the current permutation and the intervals at which it should be sorted * Then for each interval separately (except for the last one, if there is a limit) * We sort it based on data about the current column, and find all the intervals within this diff --git a/src/Common/BitonicSort.h b/src/Common/BitonicSort.h deleted file mode 100644 index 8140687c040..00000000000 --- a/src/Common/BitonicSort.h +++ /dev/null @@ -1,221 +0,0 @@ -#pragma once - -#include -#if !defined(__APPLE__) && !defined(__FreeBSD__) -#include -#endif - -#ifdef __APPLE__ -#include -#else -#include -#endif - -#include -#include -#include -#include -#include - -#include "oclBasics.h" -#include "bitonicSortKernels.cl" - -class BitonicSort -{ -public: - using KernelType = OCL::KernelType; - - enum Types - { - KernelInt8 = 0, - KernelUInt8, - KernelInt16, - KernelUInt16, - KernelInt32, - KernelUInt32, - KernelInt64, - KernelUInt64, - KernelMax - }; - - static BitonicSort & getInstance() - { - static BitonicSort instance = BitonicSort(); - return instance; - } - - /// Sorts given array in specified order. Returns `true` if given sequence was sorted, `false` otherwise. - template - bool sort(const DB::PaddedPODArray & data, DB::IColumn::Permutation & res, cl_uint sort_ascending [[maybe_unused]]) const - { - if constexpr ( - std::is_same_v || - std::is_same_v || - std::is_same_v || - std::is_same_v || - std::is_same_v || - std::is_same_v || - std::is_same_v || - std::is_same_v) - { - size_t data_size = data.size(); - - /// Getting the nearest power of 2. - size_t power = 8; - while (power < data_size) - power <<= 1; - - /// Allocates more space for additional stubs to be added if needed. - std::vector pairs_content(power); - std::vector pairs_indices(power); - - memcpy(&pairs_content[0], &data[0], sizeof(T) * data_size); - for (UInt32 i = 0; i < data_size; ++i) - pairs_indices[i] = i; - - fillWithStubs(pairs_content.data(), pairs_indices.data(), data_size, power - data_size, sort_ascending); - sort(pairs_content.data(), pairs_indices.data(), power, sort_ascending); - - for (size_t i = 0, shift = 0; i < power; ++i) - { - if (pairs_indices[i] >= data_size) - { - ++shift; - continue; - } - res[i - shift] = pairs_indices[i]; - } - - return true; - } - - return false; - } - - /// Creating a configuration instance with making all OpenCl required variables - /// such as device, platform, context, queue, program and kernel. - void configure() - { - OCL::Settings settings = OCL::Settings(1, nullptr, 1, nullptr, 1, 0); - - cl_platform_id platform = OCL::getPlatformID(settings); - cl_device_id device = OCL::getDeviceID(platform, settings); - cl_context gpu_context = OCL::makeContext(device, settings); - cl_command_queue command_queue = OCL::makeCommandQueue<2>(device, gpu_context, settings); - - cl_program program = OCL::makeProgram(bitonic_sort_kernels, gpu_context, device, settings); - - /// Creating kernels for each specified data type. - cl_int error = 0; - kernels.resize(KernelMax); - - kernels[KernelInt8] = std::shared_ptr(clCreateKernel(program, "bitonicSort_char", &error), clReleaseKernel); - OCL::checkError(error); - - kernels[KernelUInt8] = std::shared_ptr(clCreateKernel(program, "bitonicSort_uchar", &error), clReleaseKernel); - OCL::checkError(error); - - kernels[KernelInt16] = std::shared_ptr(clCreateKernel(program, "bitonicSort_short", &error), clReleaseKernel); - OCL::checkError(error); - - kernels[KernelUInt16] = std::shared_ptr(clCreateKernel(program, "bitonicSort_ushort", &error), clReleaseKernel); - OCL::checkError(error); - - kernels[KernelInt32] = std::shared_ptr(clCreateKernel(program, "bitonicSort_int", &error), clReleaseKernel); - OCL::checkError(error); - - kernels[KernelUInt32] = std::shared_ptr(clCreateKernel(program, "bitonicSort_uint", &error), clReleaseKernel); - OCL::checkError(error); - - kernels[KernelInt64] = std::shared_ptr(clCreateKernel(program, "bitonicSort_long", &error), clReleaseKernel); - OCL::checkError(error); - - kernels[KernelUInt64] = std::shared_ptr(clCreateKernel(program, "bitonicSort_ulong", &error), clReleaseKernel); - OCL::checkError(error); - - configuration = std::shared_ptr(new OCL::Configuration(device, gpu_context, command_queue, program)); - } - -private: - /// Dictionary with kernels for each type from list: uchar, char, ushort, short, uint, int, ulong and long. - std::vector> kernels; - /// Current configuration with core OpenCL instances. - std::shared_ptr configuration = nullptr; - - cl_kernel getKernel(Int8) const { return kernels[KernelInt8].get(); } - cl_kernel getKernel(UInt8) const { return kernels[KernelUInt8].get(); } - cl_kernel getKernel(Int16) const { return kernels[KernelInt16].get(); } - cl_kernel getKernel(UInt16) const { return kernels[KernelUInt16].get(); } - cl_kernel getKernel(Int32) const { return kernels[KernelInt32].get(); } - cl_kernel getKernel(UInt32) const { return kernels[KernelUInt32].get(); } - cl_kernel getKernel(Int64) const { return kernels[KernelInt64].get(); } - cl_kernel getKernel(UInt64) const { return kernels[KernelUInt64].get(); } - - /// Sorts p_input inplace with indices. Works only with arrays which size equals to power of two. - template - void sort(T * p_input, cl_uint * indices, cl_int array_size, cl_uint sort_ascending) const - { - cl_kernel kernel = getKernel(T(0)); - cl_int error = CL_SUCCESS; - cl_int num_stages = 0; - - for (cl_int temp = array_size; temp > 2; temp >>= 1) - num_stages++; - - /// Creating OpenCL buffers using input arrays memory. - cl_mem cl_input_buffer = OCL::createBuffer(p_input, array_size, configuration.get()->context()); - cl_mem cl_indices_buffer = OCL::createBuffer(indices, array_size, configuration.get()->context()); - - configureKernel(kernel, 0, static_cast(&cl_input_buffer)); - configureKernel(kernel, 1, static_cast(&cl_indices_buffer)); - configureKernel(kernel, 4, static_cast(&sort_ascending)); - - for (cl_int stage = 0; stage < num_stages; stage++) - { - configureKernel(kernel, 2, static_cast(&stage)); - - for (cl_int pass_of_stage = stage; pass_of_stage >= 0; pass_of_stage--) - { - configureKernel(kernel, 3, static_cast(&pass_of_stage)); - - /// Setting work-item dimensions. - size_t gsize = array_size / (2 * 4); - size_t global_work_size[1] = {pass_of_stage ? gsize : gsize << 1 }; // number of quad items in input array - - /// Executing kernel. - error = clEnqueueNDRangeKernel(configuration.get()->commandQueue(), kernel, 1, nullptr, - global_work_size, nullptr, 0, nullptr, nullptr); - OCL::checkError(error); - } - } - - /// Syncs all threads. - OCL::finishCommandQueue(configuration.get()->commandQueue()); - - OCL::releaseData(p_input, array_size, cl_input_buffer, configuration.get()->commandQueue()); - OCL::releaseData(indices, array_size, cl_indices_buffer, configuration.get()->commandQueue()); - } - - template - void configureKernel(cl_kernel kernel, int number_of_argument, void * source) const - { - cl_int error = clSetKernelArg(kernel, number_of_argument, sizeof(T), source); - OCL::checkError(error); - } - - /// Fills given sequences from `arraySize` index with `numberOfStubs` values. - template - void fillWithStubs(T * p_input, cl_uint * indices, cl_int array_size, cl_int number_of_stubs, cl_uint sort_ascending) const - { - T value = sort_ascending ? std::numeric_limits::max() : std::numeric_limits::min(); - for (cl_int index = 0; index < number_of_stubs; ++index) - { - p_input[array_size + index] = value; - indices[array_size + index] = array_size + index; - } - } - - BitonicSort() = default; - BitonicSort(BitonicSort const &) = delete; - void operator = (BitonicSort const &) = delete; -}; diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 85da23fb303..bf475bc9b21 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -486,7 +486,6 @@ namespace ErrorCodes extern const int NO_REMOTE_SHARD_AVAILABLE = 519; extern const int CANNOT_DETACH_DICTIONARY_AS_TABLE = 520; extern const int ATOMIC_RENAME_FAIL = 521; - extern const int OPENCL_ERROR = 522; extern const int UNKNOWN_ROW_POLICY = 523; extern const int ALTER_OF_COLUMN_IS_FORBIDDEN = 524; extern const int INCORRECT_DISK_INDEX = 525; diff --git a/src/Common/oclBasics.h b/src/Common/oclBasics.h deleted file mode 100644 index a3e7636af1b..00000000000 --- a/src/Common/oclBasics.h +++ /dev/null @@ -1,354 +0,0 @@ -#pragma once - -#include -#if USE_OPENCL - -#if !defined(__APPLE__) && !defined(__FreeBSD__) -#include -#endif - -#ifdef __APPLE__ -#include -#else -#include -#endif - -#include -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int OPENCL_ERROR; -} -} - -struct OCL -{ - using KernelType = std::remove_reference::type; - - /** - * Structure which represents the most essential settings of common OpenCl entities. - */ - struct Settings - { - // Platform info - cl_uint number_of_platform_entries; - cl_uint * number_of_available_platforms; - - // Devices info - cl_uint number_of_devices_entries; - cl_uint * number_of_available_devices; - - // Context settings - cl_context_properties * context_properties; - - void (* context_callback)(const char *, const void *, size_t, void *); - - void * context_callback_data; - - // Command queue settings - cl_command_queue_properties command_queue_properties; - - // Build settings - cl_uint number_of_program_source_pointers; - - void (* build_notification_routine)(cl_program, void *user_data); - - void * build_callback_data; - char * build_options; - - Settings(cl_uint number_of_platform_entries_, - cl_uint * number_of_available_platforms_, - cl_uint number_of_devices_entries_, - cl_uint * number_of_available_devices_, - cl_uint number_of_program_source_pointers_, - cl_command_queue_properties command_queue_properties_, - cl_context_properties * context_properties_ = nullptr, - void * context_data_callback_ = nullptr, - void (* context_callback_)(const char *, const void *, size_t, void *) = nullptr, - void (* build_notification_routine_)(cl_program, void * user_data) = nullptr, - void * build_callback_data_ = nullptr, - char * build_options_ = nullptr) - { - this->number_of_platform_entries = number_of_platform_entries_; - this->number_of_available_platforms = number_of_available_platforms_; - this->number_of_devices_entries = number_of_devices_entries_; - this->number_of_available_devices = number_of_available_devices_; - this->number_of_program_source_pointers = number_of_program_source_pointers_; - this->command_queue_properties = command_queue_properties_; - this->context_properties = context_properties_; - this->context_callback = context_callback_; - this->context_callback_data = context_data_callback_; - this->build_notification_routine = build_notification_routine_; - this->build_callback_data = build_callback_data_; - this->build_options = build_options_; - } - }; - - - /** - * Configuration with already created OpenCl common entities. - */ - class Configuration - { - public: - - Configuration(cl_device_id device, cl_context gpu_context, - cl_command_queue command_queue, cl_program program) - { - this->device_ = device; - this->gpu_context_ = std::shared_ptr(gpu_context, clReleaseContext); - this->command_queue_ = std::shared_ptr(command_queue, clReleaseCommandQueue); - this->program_ = std::shared_ptr(program, clReleaseProgram); - } - - cl_device_id device() { return device_; } - - cl_context context() { return gpu_context_.get(); } - - cl_command_queue commandQueue() { return command_queue_.get(); } - - cl_program program() { return program_.get(); } - - private: - - using ProgramType = std::remove_reference::type; - using CommandQueueType = std::remove_reference::type; - using ContextType = std::remove_reference::type; - - cl_device_id device_; - - std::shared_ptr gpu_context_; - std::shared_ptr command_queue_; - std::shared_ptr program_; - }; - - - static String opencl_error_to_str(cl_int error) - { -#define CASE_CL_CONSTANT(NAME) case NAME: return #NAME; - - // Suppose that no combinations are possible. - switch (error) - { - CASE_CL_CONSTANT(CL_SUCCESS) - CASE_CL_CONSTANT(CL_DEVICE_NOT_FOUND) - CASE_CL_CONSTANT(CL_DEVICE_NOT_AVAILABLE) - CASE_CL_CONSTANT(CL_COMPILER_NOT_AVAILABLE) - CASE_CL_CONSTANT(CL_MEM_OBJECT_ALLOCATION_FAILURE) - CASE_CL_CONSTANT(CL_OUT_OF_RESOURCES) - CASE_CL_CONSTANT(CL_OUT_OF_HOST_MEMORY) - CASE_CL_CONSTANT(CL_PROFILING_INFO_NOT_AVAILABLE) - CASE_CL_CONSTANT(CL_MEM_COPY_OVERLAP) - CASE_CL_CONSTANT(CL_IMAGE_FORMAT_MISMATCH) - CASE_CL_CONSTANT(CL_IMAGE_FORMAT_NOT_SUPPORTED) - CASE_CL_CONSTANT(CL_BUILD_PROGRAM_FAILURE) - CASE_CL_CONSTANT(CL_MAP_FAILURE) - CASE_CL_CONSTANT(CL_MISALIGNED_SUB_BUFFER_OFFSET) - CASE_CL_CONSTANT(CL_EXEC_STATUS_ERROR_FOR_EVENTS_IN_WAIT_LIST) - CASE_CL_CONSTANT(CL_COMPILE_PROGRAM_FAILURE) - CASE_CL_CONSTANT(CL_LINKER_NOT_AVAILABLE) - CASE_CL_CONSTANT(CL_LINK_PROGRAM_FAILURE) - CASE_CL_CONSTANT(CL_DEVICE_PARTITION_FAILED) - CASE_CL_CONSTANT(CL_KERNEL_ARG_INFO_NOT_AVAILABLE) - CASE_CL_CONSTANT(CL_INVALID_VALUE) - CASE_CL_CONSTANT(CL_INVALID_DEVICE_TYPE) - CASE_CL_CONSTANT(CL_INVALID_PLATFORM) - CASE_CL_CONSTANT(CL_INVALID_DEVICE) - CASE_CL_CONSTANT(CL_INVALID_CONTEXT) - CASE_CL_CONSTANT(CL_INVALID_QUEUE_PROPERTIES) - CASE_CL_CONSTANT(CL_INVALID_COMMAND_QUEUE) - CASE_CL_CONSTANT(CL_INVALID_HOST_PTR) - CASE_CL_CONSTANT(CL_INVALID_MEM_OBJECT) - CASE_CL_CONSTANT(CL_INVALID_IMAGE_FORMAT_DESCRIPTOR) - CASE_CL_CONSTANT(CL_INVALID_IMAGE_SIZE) - CASE_CL_CONSTANT(CL_INVALID_SAMPLER) - CASE_CL_CONSTANT(CL_INVALID_BINARY) - CASE_CL_CONSTANT(CL_INVALID_BUILD_OPTIONS) - CASE_CL_CONSTANT(CL_INVALID_PROGRAM) - CASE_CL_CONSTANT(CL_INVALID_PROGRAM_EXECUTABLE) - CASE_CL_CONSTANT(CL_INVALID_KERNEL_NAME) - CASE_CL_CONSTANT(CL_INVALID_KERNEL_DEFINITION) - CASE_CL_CONSTANT(CL_INVALID_KERNEL) - CASE_CL_CONSTANT(CL_INVALID_ARG_INDEX) - CASE_CL_CONSTANT(CL_INVALID_ARG_VALUE) - CASE_CL_CONSTANT(CL_INVALID_ARG_SIZE) - CASE_CL_CONSTANT(CL_INVALID_KERNEL_ARGS) - CASE_CL_CONSTANT(CL_INVALID_WORK_DIMENSION) - CASE_CL_CONSTANT(CL_INVALID_WORK_GROUP_SIZE) - CASE_CL_CONSTANT(CL_INVALID_WORK_ITEM_SIZE) - CASE_CL_CONSTANT(CL_INVALID_GLOBAL_OFFSET) - CASE_CL_CONSTANT(CL_INVALID_EVENT_WAIT_LIST) - CASE_CL_CONSTANT(CL_INVALID_EVENT) - CASE_CL_CONSTANT(CL_INVALID_OPERATION) - CASE_CL_CONSTANT(CL_INVALID_GL_OBJECT) - CASE_CL_CONSTANT(CL_INVALID_BUFFER_SIZE) - CASE_CL_CONSTANT(CL_INVALID_MIP_LEVEL) - CASE_CL_CONSTANT(CL_INVALID_GLOBAL_WORK_SIZE) - CASE_CL_CONSTANT(CL_INVALID_PROPERTY) - CASE_CL_CONSTANT(CL_INVALID_IMAGE_DESCRIPTOR) - CASE_CL_CONSTANT(CL_INVALID_COMPILER_OPTIONS) - CASE_CL_CONSTANT(CL_INVALID_LINKER_OPTIONS) - CASE_CL_CONSTANT(CL_INVALID_DEVICE_PARTITION_COUNT) - default: - return "UNKNOWN ERROR CODE "; - } - -#undef CASE_CL_CONSTANT - } - - - static void checkError(cl_int error) - { - if (error != CL_SUCCESS) - throw DB::Exception("OpenCL error: " + opencl_error_to_str(error), DB::ErrorCodes::OPENCL_ERROR); - } - - - /// Getting OpenCl main entities. - - static cl_platform_id getPlatformID(const Settings & settings) - { - cl_platform_id platform; - cl_int error = clGetPlatformIDs(settings.number_of_platform_entries, &platform, - settings.number_of_available_platforms); - checkError(error); - return platform; - } - - static cl_device_id getDeviceID(cl_platform_id & platform, const Settings & settings) - { - cl_device_id device; - cl_int error = clGetDeviceIDs(platform, CL_DEVICE_TYPE_GPU, settings.number_of_devices_entries, - &device, settings.number_of_available_devices); - OCL::checkError(error); - return device; - } - - static cl_context makeContext(cl_device_id & device, const Settings & settings) - { - cl_int error; - cl_context gpu_context = clCreateContext(settings.context_properties, settings.number_of_devices_entries, - &device, settings.context_callback, settings.context_callback_data, - &error); - OCL::checkError(error); - return gpu_context; - } - - template - static cl_command_queue makeCommandQueue(cl_device_id & device, cl_context & context, const Settings & settings [[maybe_unused]]) - { - cl_int error; - cl_command_queue command_queue; - - if constexpr (version == 1) - { -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wdeprecated-declarations" - command_queue = clCreateCommandQueue(context, device, settings.command_queue_properties, &error); -#pragma GCC diagnostic pop - } - else - { -#ifdef CL_VERSION_2_0 - command_queue = clCreateCommandQueueWithProperties(context, device, nullptr, &error); -#else - throw DB::Exception("Binary is built with OpenCL version < 2.0", DB::ErrorCodes::OPENCL_ERROR); -#endif - } - - OCL::checkError(error); - return command_queue; - } - - static cl_program makeProgram(const char * source_code, cl_context context, - cl_device_id device_id, const Settings & settings) - { - cl_int error = 0; - size_t source_size = strlen(source_code); - - cl_program program = clCreateProgramWithSource(context, settings.number_of_program_source_pointers, - &source_code, &source_size, &error); - checkError(error); - - error = clBuildProgram(program, settings.number_of_devices_entries, &device_id, settings.build_options, - settings.build_notification_routine, settings.build_callback_data); - - /// Combining additional logs output when program build failed. - if (error == CL_BUILD_PROGRAM_FAILURE) - { - size_t log_size; - error = clGetProgramBuildInfo(program, device_id, CL_PROGRAM_BUILD_LOG, 0, nullptr, &log_size); - - checkError(error); - - std::vector log(log_size); - clGetProgramBuildInfo(program, device_id, CL_PROGRAM_BUILD_LOG, log_size, log.data(), nullptr); - - checkError(error); - throw DB::Exception(log.data(), DB::ErrorCodes::OPENCL_ERROR); - } - - checkError(error); - return program; - } - - /// Configuring buffer for given input data - - template - static cl_mem createBuffer(K * p_input, cl_int array_size, cl_context context, cl_int elements_size = sizeof(K)) - { - cl_int error = CL_SUCCESS; - cl_mem cl_input_buffer = clCreateBuffer( - context, - CL_MEM_USE_HOST_PTR, - zeroCopySizeAlignment(elements_size * array_size), - p_input, - &error); - checkError(error); - return cl_input_buffer; - } - - static size_t zeroCopySizeAlignment(size_t required_size) - { - return required_size + (~required_size + 1) % 64; - } - - /// Manipulating with common OpenCL variables. - - static void finishCommandQueue(cl_command_queue command_queue) - { - // Blocks until all previously queued OpenCL commands in a queue are issued to the associated device. - cl_int error = clFinish(command_queue); - OCL::checkError(error); - } - - template - static void releaseData(T * origin, cl_int array_size, cl_mem cl_buffer, cl_command_queue command_queue, size_t offset = 0) - { - cl_int error = CL_SUCCESS; - - void * tmp_ptr = nullptr; - - // No events specified to be completed before enqueueing buffers, - // so `num_events_in_wait_list` passed with `0` value. - - tmp_ptr = clEnqueueMapBuffer(command_queue, cl_buffer, true, CL_MAP_READ, - offset, sizeof(cl_int) * array_size, 0, nullptr, nullptr, &error); - OCL::checkError(error); - if (tmp_ptr != origin) - throw DB::Exception("clEnqueueMapBuffer failed to return original pointer", DB::ErrorCodes::OPENCL_ERROR); - - error = clEnqueueUnmapMemObject(command_queue, cl_buffer, tmp_ptr, 0, nullptr, nullptr); - checkError(error); - - error = clReleaseMemObject(cl_buffer); - checkError(error); - } -}; - -#endif diff --git a/src/Common/tests/CMakeLists.txt b/src/Common/tests/CMakeLists.txt index 8de9424e044..6a39c2f8553 100644 --- a/src/Common/tests/CMakeLists.txt +++ b/src/Common/tests/CMakeLists.txt @@ -35,11 +35,6 @@ add_executable (radix_sort radix_sort.cpp) target_link_libraries (radix_sort PRIVATE clickhouse_common_io) target_include_directories(radix_sort SYSTEM PRIVATE ${PDQSORT_INCLUDE_DIR}) -if (USE_OPENCL) - add_executable (bitonic_sort bitonic_sort.cpp) - target_link_libraries (bitonic_sort PRIVATE clickhouse_common_io ${OPENCL_LINKER_FLAGS} ${OpenCL_LIBRARIES}) -endif () - add_executable (arena_with_free_lists arena_with_free_lists.cpp) target_link_libraries (arena_with_free_lists PRIVATE dbms) diff --git a/src/Common/tests/bitonic_sort.cpp b/src/Common/tests/bitonic_sort.cpp deleted file mode 100644 index 2545662c8cb..00000000000 --- a/src/Common/tests/bitonic_sort.cpp +++ /dev/null @@ -1,174 +0,0 @@ -#include -#include - -#if !defined(__APPLE__) && !defined(__FreeBSD__) -#include -#endif -#include -#include -#include -#include -#include -#include - -#include "Common/BitonicSort.h" - - -/// Generates vector of size 8 for testing. -/// Vector contains max possible value, min possible value and duplicate values. -template -static void generateTest(std::vector & data, Type min_value, Type max_value) -{ - int size = 10; - - data.resize(size); - data[0] = 10; - data[1] = max_value; - data[2] = 10; - data[3] = 20; - data[4] = min_value; - data[5] = min_value + 1; - data[6] = max_value - 5; - data[7] = 1; - data[8] = 42; - data[9] = max_value - 1; -} - - -static void check(const std::vector & indices, bool reverse = true) -{ - std::vector reference_indices{4, 5, 7, 0, 2, 3, 8, 6, 9, 1}; - if (reverse) std::reverse(reference_indices.begin(), reference_indices.end()); - - bool success = true; - for (size_t index = 0; index < reference_indices.size(); ++index) - { - if (indices[index] != reference_indices[index]) - { - success = false; - std::cerr << "Test failed. Reason: indices[" << index << "] = " - << indices[index] << ", it must be equal to " << reference_indices[index] << "\n"; - } - } - - std::string order_description = reverse ? "descending" : "ascending"; - std::cerr << "Sorted " << order_description << " sequence. Result: " << (success ? "Ok." : "Fail!") << "\n"; -} - - -template -static void sortBitonicSortWithPodArrays(const std::vector & data, std::vector & indices, bool ascending = true) -{ - DB::PaddedPODArray pod_array_data = DB::PaddedPODArray(data.size()); - DB::IColumn::Permutation pod_array_indices = DB::IColumn::Permutation(data.size()); - - for (size_t index = 0; index < data.size(); ++index) - { - *(pod_array_data.data() + index) = data[index]; - *(pod_array_indices.data() + index) = index; - } - - BitonicSort::getInstance().sort(pod_array_data, pod_array_indices, ascending); - - for (size_t index = 0; index < data.size(); ++index) - indices[index] = pod_array_indices[index]; -} - - -template -static void testBitonicSort(const std::string & test_name, Type min_value, Type max_value) -{ - std::cerr << test_name << std::endl; - - std::vector data; - generateTest(data, min_value, max_value); - - std::vector indices(data.size()); - - sortBitonicSortWithPodArrays(data, indices, true); - check(indices, false); - - sortBitonicSortWithPodArrays(data, indices, false); - check(indices, true); -} - - -static void straightforwardTests() -{ - testBitonicSort("Test 01: Int8.", CHAR_MIN, CHAR_MAX); - testBitonicSort("Test 02: UInt8.", 0, UCHAR_MAX); - testBitonicSort("Test 03: Int16.", SHRT_MIN, SHRT_MAX); - testBitonicSort("Test 04: UInt16.", 0, USHRT_MAX); - testBitonicSort("Test 05: Int32.", INT_MIN, INT_MAX); - testBitonicSort("Test 06: UInt32.", 0, UINT_MAX); - testBitonicSort("Test 07: Int64.", LONG_MIN, LONG_MAX); - testBitonicSort("Test 08: UInt64.", 0, ULONG_MAX); -} - - -template -static void bitonicSort(std::vector & data) -{ - size_t size = data.size(); - std::vector indices(size); - for (size_t i = 0; i < size; ++i) - indices[i] = i; - - sortBitonicSortWithPodArrays(data, indices); - - std::vector result(size); - for (size_t i = 0; i < size; ++i) - result[i] = data[indices[i]]; - - data = std::move(result); -} - - -template -static bool checkSort(const std::vector & data, size_t size) -{ - std::vector copy1(data.begin(), data.begin() + size); - std::vector copy2(data.begin(), data.begin() + size); - - std::sort(copy1.data(), copy1.data() + size); - bitonicSort(copy2); - - for (size_t i = 0; i < size; ++i) - if (copy1[i] != copy2[i]) - return false; - - return true; -} - - -int main() -{ - BitonicSort::getInstance().configure(); - - straightforwardTests(); - - size_t size = 1100; - std::vector data(size); - for (size_t i = 0; i < size; ++i) - data[i] = rand(); - - for (size_t i = 0; i < 128; ++i) - { - if (!checkSort(data, i)) - { - std::cerr << "fail at length " << i << std::endl; - return 1; - } - } - - for (size_t i = 128; i < size; i += 7) - { - if (!checkSort(data, i)) - { - std::cerr << "fail at length " << i << std::endl; - return 1; - } - } - - return 0; -} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5417bbad64c..b96b1b12c24 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -105,8 +105,6 @@ class IColumn; M(UInt64, parallel_replicas_count, 0, "", 0) \ M(UInt64, parallel_replica_offset, 0, "", 0) \ \ - M(SpecialSort, special_sort, SpecialSort::NOT_SPECIFIED, "Specifies a sorting algorithm which will be using in ORDER BY query.", 0) \ - \ M(Bool, skip_unavailable_shards, false, "If 1, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \ \ M(UInt64, parallel_distributed_insert_select, 0, "Process distributed INSERT SELECT query in the same cluster on local tables on every shard, if 1 SELECT is executed on each shard, if 2 SELECT and INSERT is executed on each shard", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index c0d2906e2fc..b4db51a506d 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -23,11 +23,6 @@ IMPLEMENT_SETTING_ENUM(LoadBalancing, ErrorCodes::UNKNOWN_LOAD_BALANCING, {"round_robin", LoadBalancing::ROUND_ROBIN}}) -IMPLEMENT_SETTING_ENUM(SpecialSort, ErrorCodes::UNKNOWN_JOIN, - {{"not_specified", SpecialSort::NOT_SPECIFIED}, - {"opencl_bitonic", SpecialSort::OPENCL_BITONIC}}) - - IMPLEMENT_SETTING_ENUM(JoinStrictness, ErrorCodes::UNKNOWN_JOIN, {{"", JoinStrictness::Unspecified}, {"ALL", JoinStrictness::ALL}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 7ed5ffb0c35..426497fff78 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -47,15 +47,6 @@ enum class JoinAlgorithm DECLARE_SETTING_ENUM(JoinAlgorithm) -enum class SpecialSort -{ - NOT_SPECIFIED = 0, - OPENCL_BITONIC, -}; - -DECLARE_SETTING_ENUM(SpecialSort) - - /// Which rows should be included in TOTALS. enum class TotalsMode { diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index 2198789b0b6..bd3b7bc45ff 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -32,22 +32,20 @@ struct SortColumnDescription std::shared_ptr collator; /// Collator for locale-specific comparison of strings bool with_fill; FillColumnDescription fill_description; - SpecialSort special_sort; - SortColumnDescription( size_t column_number_, int direction_, int nulls_direction_, - const std::shared_ptr & collator_ = nullptr, SpecialSort special_sort_ = SpecialSort::NOT_SPECIFIED, + const std::shared_ptr & collator_ = nullptr, bool with_fill_ = false, const FillColumnDescription & fill_description_ = {}) : column_number(column_number_), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) - , with_fill(with_fill_), fill_description(fill_description_), special_sort(special_sort_) {} + , with_fill(with_fill_), fill_description(fill_description_) {} SortColumnDescription( const std::string & column_name_, int direction_, int nulls_direction_, - const std::shared_ptr & collator_ = nullptr, SpecialSort special_sort_ = SpecialSort::NOT_SPECIFIED, + const std::shared_ptr & collator_ = nullptr, bool with_fill_ = false, const FillColumnDescription & fill_description_ = {}) : column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_) - , collator(collator_), with_fill(with_fill_), fill_description(fill_description_), special_sort(special_sort_) {} + , collator(collator_), with_fill(with_fill_), fill_description(fill_description_) {} bool operator == (const SortColumnDescription & other) const { diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8f9f22e6ee2..22106387fc4 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -621,7 +621,6 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, const Co { SortDescription order_descr; order_descr.reserve(query.orderBy()->children.size()); - SpecialSort special_sort = context.getSettings().special_sort.value; for (const auto & elem : query.orderBy()->children) { String name = elem->children.front()->getColumnName(); @@ -635,10 +634,10 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, const Co { FillColumnDescription fill_desc = getWithFillDescription(order_by_elem, context); order_descr.emplace_back(name, order_by_elem.direction, - order_by_elem.nulls_direction, collator, special_sort, true, fill_desc); + order_by_elem.nulls_direction, collator, true, fill_desc); } else - order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator, special_sort); + order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator); } return order_descr; diff --git a/src/Interpreters/sortBlock.cpp b/src/Interpreters/sortBlock.cpp index d84708b9c57..c2436806fcd 100644 --- a/src/Interpreters/sortBlock.cpp +++ b/src/Interpreters/sortBlock.cpp @@ -132,12 +132,7 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) else if (!isColumnConst(*column)) { int nan_direction_hint = description[0].nulls_direction; - auto special_sort = description[0].special_sort; - - if (special_sort == SpecialSort::OPENCL_BITONIC) - column->getSpecialPermutation(reverse, limit, nan_direction_hint, perm, IColumn::SpecialSort::OPENCL_BITONIC); - else - column->getPermutation(reverse, limit, nan_direction_hint, perm); + column->getPermutation(reverse, limit, nan_direction_hint, perm); } else /// we don't need to do anything with const column From 772eccf03d70b197f360479477405f1120f0df4c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 17:47:22 +0300 Subject: [PATCH 210/273] Move code to .cpp --- src/Interpreters/QueryParameterVisitor.cpp | 46 ++++++++++++++++++++++ src/Interpreters/QueryParameterVisitor.h | 35 +++------------- src/Interpreters/ya.make | 1 + 3 files changed, 53 insertions(+), 29 deletions(-) create mode 100644 src/Interpreters/QueryParameterVisitor.cpp diff --git a/src/Interpreters/QueryParameterVisitor.cpp b/src/Interpreters/QueryParameterVisitor.cpp new file mode 100644 index 00000000000..8a257830443 --- /dev/null +++ b/src/Interpreters/QueryParameterVisitor.cpp @@ -0,0 +1,46 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +QueryParameterVisitor::QueryParameterVisitor(NameSet & parameters_name) + : query_parameters(parameters_name) +{ +} + +void QueryParameterVisitor::visit(const ASTPtr & ast) +{ + for (const auto & child : ast->children) + { + if (const auto & query_parameter = child->as()) + visitQueryParameter(*query_parameter); + else + visit(child); + } +} + +void QueryParameterVisitor::visitQueryParameter(const ASTQueryParameter & query_parameter) +{ + query_parameters.insert(query_parameter.name); +} + +NameSet analyzeReceiveQueryParams(const std::string & query) +{ + NameSet query_params; + const char * query_begin = query.data(); + const char * query_end = query.data() + query.size(); + + ParserQuery parser(query_end, false); + ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "analyzeReceiveQueryParams", 0, 0); + QueryParameterVisitor(query_params).visit(extract_query_ast); + return query_params; +} + +} + diff --git a/src/Interpreters/QueryParameterVisitor.h b/src/Interpreters/QueryParameterVisitor.h index d3e618058c0..24ef5b08dcf 100644 --- a/src/Interpreters/QueryParameterVisitor.h +++ b/src/Interpreters/QueryParameterVisitor.h @@ -2,48 +2,25 @@ #include #include -#include -#include -#include namespace DB { +class ASTQueryParameter; + class QueryParameterVisitor { public: - QueryParameterVisitor(NameSet & parameters_name) : query_parameters(parameters_name) {} + QueryParameterVisitor(NameSet & parameters_name); - void visit(const ASTPtr & ast) - { - for (const auto & child : ast->children) - { - if (const auto & query_parameter = child->as()) - visitQueryParameter(*query_parameter); - else - visit(child); - } - } + void visit(const ASTPtr & ast); private: NameSet & query_parameters; - void visitQueryParameter(const ASTQueryParameter & query_parameter) - { - query_parameters.insert(query_parameter.name); - } + void visitQueryParameter(const ASTQueryParameter & query_parameter); }; -NameSet analyzeReceiveQueryParams(const std::string & query) -{ - NameSet query_params; - const char * query_begin = query.data(); - const char * query_end = query.data() + query.size(); - - ParserQuery parser(query_end, false); - ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "analyzeReceiveQueryParams", 0, 0); - QueryParameterVisitor(query_params).visit(extract_query_ast); - return query_params; -} +NameSet analyzeReceiveQueryParams(const std::string & query); } diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index 5f520505a8a..bd16843517f 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -126,6 +126,7 @@ SRCS( QueryAliasesVisitor.cpp QueryLog.cpp QueryNormalizer.cpp + QueryParameterVisitor.cpp QueryThreadLog.cpp RemoveInjectiveFunctionsVisitor.cpp RenameColumnVisitor.cpp From c3040327073dabc711dd22aaba6527b566f67250 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 18:02:11 +0300 Subject: [PATCH 211/273] Even more --- src/Interpreters/QueryParameterVisitor.cpp | 43 +++++++++++++--------- src/Interpreters/QueryParameterVisitor.h | 19 ++-------- 2 files changed, 29 insertions(+), 33 deletions(-) diff --git a/src/Interpreters/QueryParameterVisitor.cpp b/src/Interpreters/QueryParameterVisitor.cpp index 8a257830443..ccf51ff41ce 100644 --- a/src/Interpreters/QueryParameterVisitor.cpp +++ b/src/Interpreters/QueryParameterVisitor.cpp @@ -1,6 +1,5 @@ -#pragma once - #include +#include #include #include #include @@ -9,26 +8,36 @@ namespace DB { -QueryParameterVisitor::QueryParameterVisitor(NameSet & parameters_name) - : query_parameters(parameters_name) -{ -} +class ASTQueryParameter; -void QueryParameterVisitor::visit(const ASTPtr & ast) +class QueryParameterVisitor { - for (const auto & child : ast->children) +public: + QueryParameterVisitor(NameSet & parameters_name) + : query_parameters(parameters_name) { - if (const auto & query_parameter = child->as()) - visitQueryParameter(*query_parameter); - else - visit(child); } -} -void QueryParameterVisitor::visitQueryParameter(const ASTQueryParameter & query_parameter) -{ - query_parameters.insert(query_parameter.name); -} + void visit(const ASTPtr & ast) + { + for (const auto & child : ast->children) + { + if (const auto & query_parameter = child->as()) + visitQueryParameter(*query_parameter); + else + visit(child); + } + } + +private: + NameSet & query_parameters; + + void visitQueryParameter(const ASTQueryParameter & query_parameter) + { + query_parameters.insert(query_parameter.name); + } +}; + NameSet analyzeReceiveQueryParams(const std::string & query) { diff --git a/src/Interpreters/QueryParameterVisitor.h b/src/Interpreters/QueryParameterVisitor.h index 24ef5b08dcf..531de2ddafa 100644 --- a/src/Interpreters/QueryParameterVisitor.h +++ b/src/Interpreters/QueryParameterVisitor.h @@ -1,26 +1,13 @@ #pragma once +#include #include -#include + namespace DB { -class ASTQueryParameter; - -class QueryParameterVisitor -{ -public: - QueryParameterVisitor(NameSet & parameters_name); - - void visit(const ASTPtr & ast); - -private: - NameSet & query_parameters; - - void visitQueryParameter(const ASTQueryParameter & query_parameter); -}; - +/// Find parameters in a query and collect them into set. NameSet analyzeReceiveQueryParams(const std::string & query); } From b87ecc2ec182ecb3fbda9347c8af0aedb97f5e34 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 18:09:12 +0300 Subject: [PATCH 212/273] Even more --- src/Interpreters/QueryParameterVisitor.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/QueryParameterVisitor.cpp b/src/Interpreters/QueryParameterVisitor.cpp index ccf51ff41ce..d96209e640a 100644 --- a/src/Interpreters/QueryParameterVisitor.cpp +++ b/src/Interpreters/QueryParameterVisitor.cpp @@ -8,8 +8,6 @@ namespace DB { -class ASTQueryParameter; - class QueryParameterVisitor { public: From c7a98132ac3e3ed336ebbc7acd93ee2f5222b50a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 18:13:51 +0300 Subject: [PATCH 213/273] Slightly better code --- src/Interpreters/ReplaceQueryParameterVisitor.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index ec824ed0cce..3dabfb06770 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -1,13 +1,10 @@ -#include #include #include #include -#include #include #include #include #include -#include #include #include #include @@ -63,11 +60,14 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) data_type->deserializeAsTextEscaped(temp_column, read_buffer, format_settings); if (!read_buffer.eof()) - throw Exception("Value " + value + " cannot be parsed as " + type_name + " for query parameter '" + ast_param.name + "'" - " because it isn't parsed completely: only " + toString(read_buffer.count()) + " of " + toString(value.size()) + " bytes was parsed: " - + value.substr(0, read_buffer.count()), ErrorCodes::BAD_QUERY_PARAMETER); + throw Exception(ErrorCodes::BAD_QUERY_PARAMETER, + "Value {} cannot be parsed as {} for query parameter '{}'" + " because it isn't parsed completely: only {} of {} bytes was parsed: {}", + value, type_name, ast_param.name, read_buffer.count(), value.size(), value.substr(0, read_buffer.count())); ast = addTypeConversionToAST(std::make_shared(temp_column[0]), type_name); + + /// Keep the original alias. ast->setAlias(alias); } From 03aec83aec5847e5187d94d312a626930399951c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 19:27:37 +0300 Subject: [PATCH 214/273] Remove obsolete code from performance test --- docker/test/performance-comparison/perf.py | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index ab360ceb04f..8b1c4d623dc 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -103,18 +103,6 @@ if not args.long: print('skipped\tTest is tagged as long.') sys.exit(0) -# Check main metric to detect infinite tests. We shouldn't have such tests anymore, -# but we did in the past, and it is convenient to be able to process old tests. -main_metric_element = root.find('main_metric/*') -if main_metric_element is not None and main_metric_element.tag != 'min_time': - raise Exception('Only the min_time main metric is supported. This test uses \'{}\''.format(main_metric_element.tag)) - -# Another way to detect infinite tests. They should have an appropriate main_metric -# but sometimes they don't. -infinite_sign = root.find('.//average_speed_not_changing_for_ms') -if infinite_sign is not None: - raise Exception('Looks like the test is infinite (sign 1)') - # Print report threshold for the test if it is set. if 'max_ignored_relative_change' in root.attrib: print(f'report-threshold\t{root.attrib["max_ignored_relative_change"]}') From 9fa49a68433beda485c08cceafe0cc395753d430 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 19:29:15 +0300 Subject: [PATCH 215/273] Fix clang-tidy --- src/Interpreters/QueryParameterVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/QueryParameterVisitor.cpp b/src/Interpreters/QueryParameterVisitor.cpp index d96209e640a..297bc211712 100644 --- a/src/Interpreters/QueryParameterVisitor.cpp +++ b/src/Interpreters/QueryParameterVisitor.cpp @@ -11,7 +11,7 @@ namespace DB class QueryParameterVisitor { public: - QueryParameterVisitor(NameSet & parameters_name) + explicit QueryParameterVisitor(NameSet & parameters_name) : query_parameters(parameters_name) { } From 7ae4d0174406a70e6d3a19de0bf96d92e2e84fa4 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 19 Sep 2020 19:31:08 +0300 Subject: [PATCH 216/273] Update convertMySQLDataType.cpp --- src/DataTypes/convertMySQLDataType.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/DataTypes/convertMySQLDataType.cpp b/src/DataTypes/convertMySQLDataType.cpp index ef6fb7de874..c67f90e6408 100644 --- a/src/DataTypes/convertMySQLDataType.cpp +++ b/src/DataTypes/convertMySQLDataType.cpp @@ -105,11 +105,10 @@ DataTypePtr convertMySQLDataType(MultiEnum type_support, else if (precision <= DecimalUtils::maxPrecision()) res = std::make_shared>(precision, scale); } - else - { - /// Also String is fallback for all unknown types. + + /// Also String is fallback for all unknown types. + if (!res) res = std::make_shared(); - } if (is_nullable) res = std::make_shared(res); From 35e8a895b646c4aa7266e5beb98e000a8038a143 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 20:32:13 +0300 Subject: [PATCH 217/273] Calculate profiler data in separate query runs --- docker/test/performance-comparison/Dockerfile | 2 +- .../users.d/perf-comparison-tweaks-users.xml | 2 - docker/test/performance-comparison/perf.py | 46 ++++++++++++++++--- 3 files changed, 41 insertions(+), 9 deletions(-) diff --git a/docker/test/performance-comparison/Dockerfile b/docker/test/performance-comparison/Dockerfile index df666af8e8e..a4f8af2f388 100644 --- a/docker/test/performance-comparison/Dockerfile +++ b/docker/test/performance-comparison/Dockerfile @@ -29,7 +29,7 @@ RUN apt-get update \ tzdata \ vim \ wget \ - && pip3 --no-cache-dir install clickhouse_driver \ + && pip3 --no-cache-dir install clickhouse_driver scipy \ && apt-get purge --yes python3-dev g++ \ && apt-get autoremove --yes \ && apt-get clean \ diff --git a/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml b/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml index c6d9f7ea582..cee7dc3ff16 100644 --- a/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml +++ b/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml @@ -1,8 +1,6 @@ - 10000000 - 0 1 1 1 diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 8b1c4d623dc..cc55e1de5f9 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -15,6 +15,7 @@ import sys import time import traceback import xml.etree.ElementTree as et +from scipy import stats def tsv_escape(s): return s.replace('\\', '\\\\').replace('\t', '\\t').replace('\n', '\\n').replace('\r','') @@ -211,7 +212,8 @@ for query_index in queries_to_run: for conn_index, c in enumerate(all_connections): try: prewarm_id = f'{query_prefix}.prewarm0' - res = c.execute(q, query_id = prewarm_id) + # Will also detect too long queries during warmup stage + res = c.execute(q, query_id = prewarm_id, settings = {'max_execution_time': 10}) print(f'prewarm\t{query_index}\t{prewarm_id}\t{conn_index}\t{c.last_query.elapsed}') except KeyboardInterrupt: raise @@ -221,7 +223,6 @@ for query_index in queries_to_run: query_error_on_connection[conn_index] = traceback.format_exc(); continue - # Report all errors that ocurred during prewarm and decide what to do next. # If prewarm fails for the query on all servers -- skip the query and # continue testing the next query. @@ -246,7 +247,12 @@ for query_index in queries_to_run: # sending excessive data. start_seconds = time.perf_counter() server_seconds = 0 + profile_seconds = 0 run = 0 + + # Arrays of run times for each connection. + all_server_times = [[]] * len(this_query_connections) + while True: run_id = f'{query_prefix}.run{run}' @@ -259,14 +265,17 @@ for query_index in queries_to_run: e.message = run_id + ': ' + e.message raise - server_seconds += c.last_query.elapsed - print(f'query\t{query_index}\t{run_id}\t{conn_index}\t{c.last_query.elapsed}') + elapsed = c.last_query.elapsed + all_server_times[conn_index].append(elapsed) - if c.last_query.elapsed > 10: + server_seconds += elapsed + print(f'query\t{query_index}\t{run_id}\t{conn_index}\t{elapsed}') + + if 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) + print(f'The query no. {query_index} is taking too long to run ({c.elapsed} s)', file=sys.stderr) exit(2) # Be careful with the counter, after this line it's the next iteration @@ -297,6 +306,31 @@ for query_index in queries_to_run: client_seconds = time.perf_counter() - start_seconds print(f'client-time\t{query_index}\t{client_seconds}\t{server_seconds}') + # Run additional profiling queries to collect profile data, but only if test times appeared to be different. + # We have to do it after normal runs because otherwise it will affect test statistics too much + if len(all_server_times) == 2 and stats.ttest_ind(all_server_times[0], all_server_times[1], equal_var = False).pvalue < 0.1: + while True: + run_id = f'{query_prefix}.profile{run}' + + for conn_index, c in enumerate(this_query_connections): + try: + res = c.execute(q, query_id = run_id, settings = {'query_profiler_real_time_period_ns': 10000000}) + except Exception as e: + # Add query id to the exception to make debugging easier. + e.args = (run_id, *e.args) + e.message = run_id + ': ' + e.message + raise + + elapsed = c.last_query.elapsed + profile_seconds += elapsed + + # Don't spend too much time for profile runs + if run >= args.runs or profile_seconds > 10: + continue + + run += 1 + # And don't bother with short queries + # Run drop queries drop_queries = substitute_parameters(drop_query_templates) for conn_index, c in enumerate(all_connections): From ef800b508351108e59769a3587c5c1d0ff6dbab3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 21:21:29 +0300 Subject: [PATCH 218/273] Perf test: development --- docker/test/performance-comparison/perf.py | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index cc55e1de5f9..a37ef553cfb 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -251,7 +251,9 @@ for query_index in queries_to_run: run = 0 # Arrays of run times for each connection. - all_server_times = [[]] * len(this_query_connections) + all_server_times = [] + for conn_index, c in enumerate(this_query_connections): + all_server_times.append([]) while True: run_id = f'{query_prefix}.run{run}' @@ -306,15 +308,20 @@ for query_index in queries_to_run: client_seconds = time.perf_counter() - start_seconds print(f'client-time\t{query_index}\t{client_seconds}\t{server_seconds}') + #print(all_server_times) + #print(stats.ttest_ind(all_server_times[0], all_server_times[1], equal_var = False).pvalue) + # Run additional profiling queries to collect profile data, but only if test times appeared to be different. # We have to do it after normal runs because otherwise it will affect test statistics too much if len(all_server_times) == 2 and stats.ttest_ind(all_server_times[0], all_server_times[1], equal_var = False).pvalue < 0.1: + run = 0 while True: run_id = f'{query_prefix}.profile{run}' for conn_index, c in enumerate(this_query_connections): try: res = c.execute(q, query_id = run_id, settings = {'query_profiler_real_time_period_ns': 10000000}) + print(f'profile\t{query_index}\t{run_id}\t{conn_index}\t{c.last_query.elapsed}') except Exception as e: # Add query id to the exception to make debugging easier. e.args = (run_id, *e.args) @@ -324,11 +331,10 @@ for query_index in queries_to_run: elapsed = c.last_query.elapsed profile_seconds += elapsed - # Don't spend too much time for profile runs - if run >= args.runs or profile_seconds > 10: - continue - run += 1 + # Don't spend too much time for profile runs + if run > args.runs or profile_seconds > 10: + break # And don't bother with short queries # Run drop queries From dae3b40a55ef9bb7ea47692ec882fb8a55677ce6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 22:15:16 +0300 Subject: [PATCH 219/273] Fix buffer overflow in "bar" function --- src/Common/UnicodeBar.cpp | 70 +++++++++++++++++++++++++++++++++++++++ src/Common/UnicodeBar.h | 52 +++-------------------------- src/Common/ya.make | 1 + 3 files changed, 75 insertions(+), 48 deletions(-) create mode 100644 src/Common/UnicodeBar.cpp diff --git a/src/Common/UnicodeBar.cpp b/src/Common/UnicodeBar.cpp new file mode 100644 index 00000000000..8ff5e2052c1 --- /dev/null +++ b/src/Common/UnicodeBar.cpp @@ -0,0 +1,70 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + namespace ErrorCodes + { + extern const int PARAMETER_OUT_OF_BOUND; + } +} + + +namespace UnicodeBar +{ + double getWidth(Int64 x, Int64 min, Int64 max, double max_width) + { + if (x <= min) + return 0; + + if (x >= max) + return max_width; + + /// The case when max - min overflows + Int64 max_difference; + if (common::subOverflow(max, min, max_difference)) + throw DB::Exception(DB::ErrorCodes::PARAMETER_OUT_OF_BOUND, "The arguments to render unicode bar will lead to arithmetic overflow"); + + return (x - min) * max_width / max_difference; + } + + size_t getWidthInBytes(double width) + { + return ceil(width - 1.0 / 8) * UNICODE_BAR_CHAR_SIZE; + } + + void render(double width, char * dst) + { + size_t floor_width = floor(width); + + for (size_t i = 0; i < floor_width; ++i) + { + memcpy(dst, "█", UNICODE_BAR_CHAR_SIZE); + dst += UNICODE_BAR_CHAR_SIZE; + } + + size_t remainder = floor((width - floor_width) * 8); + + if (remainder) + { + memcpy(dst, &"▏▎▍▌▋▋▊▉"[(remainder - 1) * UNICODE_BAR_CHAR_SIZE], UNICODE_BAR_CHAR_SIZE); + dst += UNICODE_BAR_CHAR_SIZE; + } + + *dst = 0; + } + + std::string render(double width) + { + std::string res(getWidthInBytes(width), '\0'); + render(width, res.data()); + return res; + } +} + diff --git a/src/Common/UnicodeBar.h b/src/Common/UnicodeBar.h index 9a5bcecbd62..0c62bd7e8f7 100644 --- a/src/Common/UnicodeBar.h +++ b/src/Common/UnicodeBar.h @@ -1,7 +1,5 @@ #pragma once -#include -#include #include #include @@ -10,54 +8,12 @@ /** Allows you to draw a unicode-art bar whose width is displayed with a resolution of 1/8 character. */ - - namespace UnicodeBar { - using DB::Int64; - - inline double getWidth(Int64 x, Int64 min, Int64 max, double max_width) - { - if (x <= min) - return 0; - - if (x >= max) - return max_width; - - return (x - min) * max_width / (max - min); - } - - inline size_t getWidthInBytes(double width) - { - return ceil(width - 1.0 / 8) * UNICODE_BAR_CHAR_SIZE; - } + double getWidth(Int64 x, Int64 min, Int64 max, double max_width); + size_t getWidthInBytes(double width); /// In `dst` there must be a space for barWidthInBytes(width) characters and a trailing zero. - inline void render(double width, char * dst) - { - size_t floor_width = floor(width); - - for (size_t i = 0; i < floor_width; ++i) - { - memcpy(dst, "█", UNICODE_BAR_CHAR_SIZE); - dst += UNICODE_BAR_CHAR_SIZE; - } - - size_t remainder = floor((width - floor_width) * 8); - - if (remainder) - { - memcpy(dst, &"▏▎▍▌▋▋▊▉"[(remainder - 1) * UNICODE_BAR_CHAR_SIZE], UNICODE_BAR_CHAR_SIZE); - dst += UNICODE_BAR_CHAR_SIZE; - } - - *dst = 0; - } - - inline std::string render(double width) - { - std::string res(getWidthInBytes(width), '\0'); - render(width, res.data()); - return res; - } + void render(double width, char * dst); + std::string render(double width); } diff --git a/src/Common/ya.make b/src/Common/ya.make index 9b671e7b9ee..fb04ecaa141 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -99,6 +99,7 @@ SRCS( ThreadProfileEvents.cpp ThreadStatus.cpp TraceCollector.cpp + UnicodeBar.cpp UTF8Helpers.cpp WeakHash.cpp ZooKeeper/IKeeper.cpp From ace6130585c53643fe11bc87d2c423b5671e5faf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 22:28:43 +0300 Subject: [PATCH 220/273] Added a test --- tests/queries/0_stateless/01502_bar_overflow.reference | 0 tests/queries/0_stateless/01502_bar_overflow.sql | 1 + 2 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/01502_bar_overflow.reference create mode 100644 tests/queries/0_stateless/01502_bar_overflow.sql diff --git a/tests/queries/0_stateless/01502_bar_overflow.reference b/tests/queries/0_stateless/01502_bar_overflow.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01502_bar_overflow.sql b/tests/queries/0_stateless/01502_bar_overflow.sql new file mode 100644 index 00000000000..cb3de7ac20b --- /dev/null +++ b/tests/queries/0_stateless/01502_bar_overflow.sql @@ -0,0 +1 @@ +SELECT bar((greatCircleAngle(100, -1, number, number) - number) * 2, -9223372036854775808, 1023, 100) FROM numbers(1048575); -- { serverError 12 } From 5a47e64685075ea50e5f11b00bda5735e8fd88bf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 22:34:24 +0300 Subject: [PATCH 221/273] Proper exception message for wrong number of arguments of CAST --- src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp | 3 +++ .../queries/0_stateless/01503_if_const_optimization.reference | 0 tests/queries/0_stateless/01503_if_const_optimization.sql | 1 + 3 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/01503_if_const_optimization.reference create mode 100644 tests/queries/0_stateless/01503_if_const_optimization.sql diff --git a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp index dd63093493f..765e7b1fa3d 100644 --- a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp +++ b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp @@ -33,6 +33,9 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v { if (const auto * expr_list = function->arguments->as()) { + if (expr_list->children.size() != 2) + throw Exception("Function CAST must have exactly two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + const ASTPtr & type_ast = expr_list->children.at(1); if (const auto * type_literal = type_ast->as()) { diff --git a/tests/queries/0_stateless/01503_if_const_optimization.reference b/tests/queries/0_stateless/01503_if_const_optimization.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01503_if_const_optimization.sql b/tests/queries/0_stateless/01503_if_const_optimization.sql new file mode 100644 index 00000000000..047f6f757e8 --- /dev/null +++ b/tests/queries/0_stateless/01503_if_const_optimization.sql @@ -0,0 +1 @@ +SELECT if(CAST(NULL), '2.55', NULL) AS x; -- { serverError 42 } From 0029c5512d03d8cebedb3e9a07f350e60255e5e7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 22:43:26 +0300 Subject: [PATCH 222/273] Maybe fix MSan report in base64 --- src/Functions/FunctionBase64Conversion.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Functions/FunctionBase64Conversion.h b/src/Functions/FunctionBase64Conversion.h index 450bad496cd..b6217f9b0bc 100644 --- a/src/Functions/FunctionBase64Conversion.h +++ b/src/Functions/FunctionBase64Conversion.h @@ -4,6 +4,7 @@ #if USE_BASE64 # include +# include # include # include # include @@ -151,6 +152,10 @@ public: } } + /// Base64 library is using AVX-512 with some shuffle operations. + /// Memory sanitizer don't understand if there was uninitialized memory in SIMD register but it was not used in the result of shuffle. + __msan_unpoison(dst_pos, outlen); + source += srclen + 1; dst_pos += outlen + 1; From 71568130d8fe84c2b54d8411dc8918d969fdc814 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sat, 19 Sep 2020 18:33:02 -0300 Subject: [PATCH 223/273] more tests for #14646 --- .../01475_mutation_with_if.reference | 17 ++++++++ .../0_stateless/01475_mutation_with_if.sql | 39 +++++++++++++++++++ 2 files changed, 56 insertions(+) diff --git a/tests/queries/0_stateless/01475_mutation_with_if.reference b/tests/queries/0_stateless/01475_mutation_with_if.reference index 2874a18147f..fed47882f5c 100644 --- a/tests/queries/0_stateless/01475_mutation_with_if.reference +++ b/tests/queries/0_stateless/01475_mutation_with_if.reference @@ -1 +1,18 @@ 1 150 +2020-02-28 car +2020-03-28 dog +2020-03-28 dog +2020-08-02 car +\N cat +2020-08-03 car +2020-03-28 dog +2020-08-02 car +2020-03-28 dog +2020-08-04 cat +2020-08-04 car +2020-08-04 car +\N cat +2020-08-05 \N +\N car +\N dog +\N car diff --git a/tests/queries/0_stateless/01475_mutation_with_if.sql b/tests/queries/0_stateless/01475_mutation_with_if.sql index 6f0ef8924be..e2207846e03 100644 --- a/tests/queries/0_stateless/01475_mutation_with_if.sql +++ b/tests/queries/0_stateless/01475_mutation_with_if.sql @@ -14,3 +14,42 @@ ALTER TABLE mutation_table UPDATE price = 150 WHERE id = 1 SETTINGS mutations_sy SELECT * FROM mutation_table; DROP TABLE IF EXISTS mutation_table; + + +create table mutation_table ( dt Nullable(Date), name Nullable(String)) +engine MergeTree order by tuple(); + +insert into mutation_table (name, dt) values ('car', '2020-02-28'); +insert into mutation_table (name, dt) values ('dog', '2020-03-28'); + +select * from mutation_table; + +alter table mutation_table update dt = toDateOrNull('2020-08-02') +where name = 'car' SETTINGS mutations_sync = 2; + +select * from mutation_table; + +insert into mutation_table (name, dt) values ('car', Null); +insert into mutation_table (name, dt) values ('cat', Null); + +alter table mutation_table update dt = toDateOrNull('2020-08-03') +where name = 'car' and dt is null SETTINGS mutations_sync = 2; + +select * from mutation_table; + +alter table mutation_table update dt = toDateOrNull('2020-08-04') +where name = 'car' or dt is null SETTINGS mutations_sync = 2; + +select * from mutation_table; + +insert into mutation_table (name, dt) values (Null, '2020-08-05'); + +alter table mutation_table update dt = Null +where name is not null SETTINGS mutations_sync = 2; + +select * from mutation_table; + + +DROP TABLE IF EXISTS mutation_table; + + From cd53a7016333368ea6f31e03bdfcf13aa41cc52e Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sat, 19 Sep 2020 18:40:14 -0300 Subject: [PATCH 224/273] constant output order --- .../0_stateless/01475_mutation_with_if.reference | 14 +++++++------- .../queries/0_stateless/01475_mutation_with_if.sql | 13 ++++++------- 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/01475_mutation_with_if.reference b/tests/queries/0_stateless/01475_mutation_with_if.reference index fed47882f5c..2d6f3e8097a 100644 --- a/tests/queries/0_stateless/01475_mutation_with_if.reference +++ b/tests/queries/0_stateless/01475_mutation_with_if.reference @@ -3,16 +3,16 @@ 2020-03-28 dog 2020-03-28 dog 2020-08-02 car -\N cat -2020-08-03 car 2020-03-28 dog 2020-08-02 car -2020-03-28 dog -2020-08-04 cat -2020-08-04 car -2020-08-04 car +2020-08-03 car \N cat +2020-03-28 dog +2020-08-04 car +2020-08-04 car +2020-08-04 cat 2020-08-05 \N \N car -\N dog \N car +\N cat +\N dog diff --git a/tests/queries/0_stateless/01475_mutation_with_if.sql b/tests/queries/0_stateless/01475_mutation_with_if.sql index e2207846e03..c25d208e924 100644 --- a/tests/queries/0_stateless/01475_mutation_with_if.sql +++ b/tests/queries/0_stateless/01475_mutation_with_if.sql @@ -16,18 +16,19 @@ SELECT * FROM mutation_table; DROP TABLE IF EXISTS mutation_table; + create table mutation_table ( dt Nullable(Date), name Nullable(String)) engine MergeTree order by tuple(); insert into mutation_table (name, dt) values ('car', '2020-02-28'); insert into mutation_table (name, dt) values ('dog', '2020-03-28'); -select * from mutation_table; +select * from mutation_table order by dt, name; alter table mutation_table update dt = toDateOrNull('2020-08-02') where name = 'car' SETTINGS mutations_sync = 2; -select * from mutation_table; +select * from mutation_table order by dt, name; insert into mutation_table (name, dt) values ('car', Null); insert into mutation_table (name, dt) values ('cat', Null); @@ -35,21 +36,19 @@ insert into mutation_table (name, dt) values ('cat', Null); alter table mutation_table update dt = toDateOrNull('2020-08-03') where name = 'car' and dt is null SETTINGS mutations_sync = 2; -select * from mutation_table; +select * from mutation_table order by dt, name; alter table mutation_table update dt = toDateOrNull('2020-08-04') where name = 'car' or dt is null SETTINGS mutations_sync = 2; -select * from mutation_table; +select * from mutation_table order by dt, name; insert into mutation_table (name, dt) values (Null, '2020-08-05'); alter table mutation_table update dt = Null where name is not null SETTINGS mutations_sync = 2; -select * from mutation_table; +select * from mutation_table order by dt, name; DROP TABLE IF EXISTS mutation_table; - - From fcee786320da9f76b4b31bac1b93dd4a88398826 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 20 Sep 2020 11:53:29 +0800 Subject: [PATCH 225/273] Fix incorrect key condition of fixed strings. --- src/Interpreters/convertFieldToType.cpp | 13 +++++++++++++ .../01503_fixed_string_primary_key.reference | 1 + .../0_stateless/01503_fixed_string_primary_key.sql | 7 +++++++ 3 files changed, 21 insertions(+) create mode 100644 tests/queries/0_stateless/01503_fixed_string_primary_key.reference create mode 100644 tests/queries/0_stateless/01503_fixed_string_primary_key.sql diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 2d624922d2a..c0e9d7edc13 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -191,7 +191,20 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID else if (which_type.isStringOrFixedString()) { if (src.getType() == Field::Types::String) + { + if (which_type.isFixedString()) + { + size_t n = assert_cast(type).getN(); + const auto & src_str = src.get(); + if (src_str.size() < n) + { + String src_str_extended = src_str; + src_str_extended.resize(n); + return src_str_extended; + } + } return src; + } } else if (const DataTypeArray * type_array = typeid_cast(&type)) { diff --git a/tests/queries/0_stateless/01503_fixed_string_primary_key.reference b/tests/queries/0_stateless/01503_fixed_string_primary_key.reference new file mode 100644 index 00000000000..45a4fb75db8 --- /dev/null +++ b/tests/queries/0_stateless/01503_fixed_string_primary_key.reference @@ -0,0 +1 @@ +8 diff --git a/tests/queries/0_stateless/01503_fixed_string_primary_key.sql b/tests/queries/0_stateless/01503_fixed_string_primary_key.sql new file mode 100644 index 00000000000..09576777cb6 --- /dev/null +++ b/tests/queries/0_stateless/01503_fixed_string_primary_key.sql @@ -0,0 +1,7 @@ +DROP TABLE IF EXISTS test; + +CREATE TABLE test(key FixedString(10)) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY (key); +INSERT INTO test SELECT toString(intDiv(number, 8)) FROM numbers(100); +SELECT count() FROM test WHERE key = '1'; + +DROP TABLE IF EXISTS test; From 8ca4d6634eea9557b9fd8562ba94ac5840a92e8d Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 20 Sep 2020 15:16:46 +0800 Subject: [PATCH 226/273] better fixed string group by support --- src/Interpreters/Aggregator.cpp | 16 ++++++++-------- src/Interpreters/Aggregator.h | 8 ++++---- .../performance/single_fixed_string_groupby.xml | 17 +++++++++++++++++ 3 files changed, 29 insertions(+), 12 deletions(-) create mode 100644 tests/performance/single_fixed_string_groupby.xml diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 86a33dccb53..1df76f96663 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -367,6 +367,14 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod() throw Exception("Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.", ErrorCodes::LOGICAL_ERROR); } + if (params.keys_size == 1 && isFixedString(types_removed_nullable[0])) + { + if (has_low_cardinality) + return AggregatedDataVariants::Type::low_cardinality_key_fixed_string; + else + return AggregatedDataVariants::Type::key_fixed_string; + } + /// If all keys fits in N bits, will use hash table with all keys packed (placed contiguously) to single N-bit key. if (params.keys_size == num_fixed_contiguous_keys) { @@ -399,14 +407,6 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod() return AggregatedDataVariants::Type::key_string; } - if (params.keys_size == 1 && isFixedString(types_removed_nullable[0])) - { - if (has_low_cardinality) - return AggregatedDataVariants::Type::low_cardinality_key_fixed_string; - else - return AggregatedDataVariants::Type::key_fixed_string; - } - return AggregatedDataVariants::Type::serialized; } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 8dce011c435..fac7889d6fc 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -228,7 +228,7 @@ struct AggregationMethodString static void insertKeyIntoColumns(const StringRef & key, MutableColumns & key_columns, const Sizes &) { - key_columns[0]->insertData(key.data, key.size); + static_cast(key_columns[0].get())->insertData(key.data, key.size); } }; @@ -254,7 +254,7 @@ struct AggregationMethodStringNoCache static void insertKeyIntoColumns(const StringRef & key, MutableColumns & key_columns, const Sizes &) { - key_columns[0]->insertData(key.data, key.size); + static_cast(key_columns[0].get())->insertData(key.data, key.size); } }; @@ -280,7 +280,7 @@ struct AggregationMethodFixedString static void insertKeyIntoColumns(const StringRef & key, MutableColumns & key_columns, const Sizes &) { - key_columns[0]->insertData(key.data, key.size); + static_cast(key_columns[0].get())->insertData(key.data, key.size); } }; @@ -305,7 +305,7 @@ struct AggregationMethodFixedStringNoCache static void insertKeyIntoColumns(const StringRef & key, MutableColumns & key_columns, const Sizes &) { - key_columns[0]->insertData(key.data, key.size); + static_cast(key_columns[0].get())->insertData(key.data, key.size); } }; diff --git a/tests/performance/single_fixed_string_groupby.xml b/tests/performance/single_fixed_string_groupby.xml new file mode 100644 index 00000000000..10ca5f1ebcd --- /dev/null +++ b/tests/performance/single_fixed_string_groupby.xml @@ -0,0 +1,17 @@ + + DROP TABLE IF EXISTS perf_lc_fixed_str_groupby + CREATE TABLE perf_lc_fixed_str_groupby( + a LowCardinality(FixedString(14)), + b LowCardinality(FixedString(14)) + ) ENGINE MergeTree ORDER BY tuple() + + + + INSERT INTO perf_lc_fixed_str_groupby SELECT ('number key ' || toString(number % 400)) AS a, ('number key ' || toString(number % 20)) AS b FROM numbers(30000000) + + + SELECT count() FROM perf_lc_fixed_str_groupby GROUP BY a + SELECT count() FROM perf_lc_fixed_str_groupby GROUP BY b + + DROP TABLE IF EXISTS perf_lc_fixed_str_groupby + From 5ffee8808a2c53c0528cd75d0128ffa24f368ff1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 20 Sep 2020 16:27:33 +0300 Subject: [PATCH 227/273] Alter remove column properties and TTLs (#14742) --- src/Databases/DatabaseOrdinary.cpp | 2 + src/Interpreters/InterpreterAlterQuery.cpp | 5 + src/Parsers/ASTAlterQuery.cpp | 21 ++- src/Parsers/ASTAlterQuery.h | 4 + src/Parsers/ParserAlterQuery.cpp | 46 ++++- src/Parsers/ParserCreateQuery.h | 28 ++- src/Storages/AlterCommands.cpp | 177 ++++++++++++++++-- src/Storages/AlterCommands.h | 31 ++- src/Storages/StorageReplicatedMergeTree.cpp | 22 ++- ...ter_remove_no_property_zookeeper.reference | 4 + ...493_alter_remove_no_property_zookeeper.sql | 51 +++++ .../01493_alter_remove_properties.reference | 20 ++ .../01493_alter_remove_properties.sql | 72 +++++++ ...lter_remove_properties_zookeeper.reference | 21 +++ ...1493_alter_remove_properties_zookeeper.sql | 92 +++++++++ ...01493_alter_remove_wrong_default.reference | 1 + .../01493_alter_remove_wrong_default.sql | 22 +++ 17 files changed, 570 insertions(+), 49 deletions(-) create mode 100644 tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.reference create mode 100644 tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.sql create mode 100644 tests/queries/0_stateless/01493_alter_remove_properties.reference create mode 100644 tests/queries/0_stateless/01493_alter_remove_properties.sql create mode 100644 tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.reference create mode 100644 tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql create mode 100644 tests/queries/0_stateless/01493_alter_remove_wrong_default.reference create mode 100644 tests/queries/0_stateless/01493_alter_remove_wrong_default.sql diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 0512a155418..13aeb7de148 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -291,6 +291,8 @@ void DatabaseOrdinary::alterTable(const Context & context, const StorageID & tab if (metadata.table_ttl.definition_ast) storage_ast.set(storage_ast.ttl_table, metadata.table_ttl.definition_ast); + else if (storage_ast.ttl_table != nullptr) /// TTL was removed + storage_ast.ttl_table = nullptr; if (metadata.settings_changes) storage_ast.set(storage_ast.settings, metadata.settings_changes); diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 8cf581eb463..73251cad991 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -222,6 +222,11 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS required_access.emplace_back(AccessType::ALTER_TTL, database, table); break; } + case ASTAlterCommand::REMOVE_TTL: + { + required_access.emplace_back(AccessType::ALTER_TTL, database, table); + break; + } case ASTAlterCommand::MATERIALIZE_TTL: { required_access.emplace_back(AccessType::ALTER_MATERIALIZE_TTL, database, table); diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index d033cdc79a2..d74156d11d8 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -99,12 +99,19 @@ void ASTAlterCommand::formatImpl( settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); col_decl->formatImpl(settings, state, frame); - if (first) - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " FIRST " << (settings.hilite ? hilite_none : ""); - else if (column) /// AFTER + if (!remove_property.empty()) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " AFTER " << (settings.hilite ? hilite_none : ""); - column->formatImpl(settings, state, frame); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " REMOVE " << remove_property; + } + else + { + if (first) + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " FIRST " << (settings.hilite ? hilite_none : ""); + else if (column) /// AFTER + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " AFTER " << (settings.hilite ? hilite_none : ""); + column->formatImpl(settings, state, frame); + } } } else if (type == ASTAlterCommand::COMMENT_COLUMN) @@ -280,6 +287,10 @@ void ASTAlterCommand::formatImpl( settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY TTL " << (settings.hilite ? hilite_none : ""); ttl->formatImpl(settings, state, frame); } + else if (type == ASTAlterCommand::REMOVE_TTL) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REMOVE TTL" << (settings.hilite ? hilite_none : ""); + } else if (type == ASTAlterCommand::MATERIALIZE_TTL) { settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MATERIALIZE TTL" diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index df27ba0a3b0..78e0c726ddf 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -36,6 +36,7 @@ public: MATERIALIZE_TTL, MODIFY_SETTING, MODIFY_QUERY, + REMOVE_TTL, ADD_INDEX, DROP_INDEX, @@ -167,6 +168,9 @@ public: /// Target column name ASTPtr rename_to; + /// Which property user want to remove + String remove_property; + String getID(char delim) const override { return "AlterCommand" + (delim + std::to_string(static_cast(type))); } ASTPtr clone() const override; diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 9930bb649b4..0739f24a688 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -82,12 +82,23 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_where("WHERE"); ParserKeyword s_to("TO"); + ParserKeyword s_remove("REMOVE"); + ParserKeyword s_default("DEFAULT"); + ParserKeyword s_materialized("MATERIALIZED"); + ParserKeyword s_alias("ALIAS"); + ParserKeyword s_comment("COMMENT"); + ParserKeyword s_codec("CODEC"); + ParserKeyword s_ttl("TTL"); + + ParserKeyword s_remove_ttl("REMOVE TTL"); + ParserCompoundIdentifier parser_name; ParserStringLiteral parser_string_literal; + ParserIdentifier parser_remove_property; ParserCompoundColumnDeclaration parser_col_decl; ParserIndexDeclaration parser_idx_decl; ParserConstraintDeclaration parser_constraint_decl; - ParserCompoundColumnDeclaration parser_modify_col_decl(false); + ParserCompoundColumnDeclaration parser_modify_col_decl(false, false, true); ParserPartition parser_partition; ParserExpression parser_exp_elem; ParserList parser_assignment_list( @@ -433,14 +444,33 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!parser_modify_col_decl.parse(pos, command->col_decl, expected)) return false; - if (s_first.ignore(pos, expected)) - command->first = true; - else if (s_after.ignore(pos, expected)) + if (s_remove.ignore(pos, expected)) { - if (!parser_name.parse(pos, command->column, expected)) + if (s_default.ignore(pos, expected)) + command->remove_property = "DEFAULT"; + else if (s_materialized.ignore(pos, expected)) + command->remove_property = "MATERIALIZED"; + else if (s_alias.ignore(pos, expected)) + command->remove_property = "ALIAS"; + else if (s_comment.ignore(pos, expected)) + command->remove_property = "COMMENT"; + else if (s_codec.ignore(pos, expected)) + command->remove_property = "CODEC"; + else if (s_ttl.ignore(pos, expected)) + command->remove_property = "TTL"; + else return false; } - + else + { + if (s_first.ignore(pos, expected)) + command->first = true; + else if (s_after.ignore(pos, expected)) + { + if (!parser_name.parse(pos, command->column, expected)) + return false; + } + } command->type = ASTAlterCommand::MODIFY_COLUMN; } else if (s_modify_order_by.ignore(pos, expected)) @@ -496,6 +526,10 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; command->type = ASTAlterCommand::MODIFY_TTL; } + else if (s_remove_ttl.ignore(pos, expected)) + { + command->type = ASTAlterCommand::REMOVE_TTL; + } else if (s_materialize_ttl.ignore(pos, expected)) { command->type = ASTAlterCommand::MATERIALIZE_TTL; diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 953f657a04e..60933f7384d 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -90,8 +90,10 @@ template class IParserColumnDeclaration : public IParserBase { public: - explicit IParserColumnDeclaration(bool require_type_ = true, bool allow_null_modifiers_ = false) - : require_type(require_type_), allow_null_modifiers(allow_null_modifiers_) + explicit IParserColumnDeclaration(bool require_type_ = true, bool allow_null_modifiers_ = false, bool check_keywords_after_name_ = false) + : require_type(require_type_) + , allow_null_modifiers(allow_null_modifiers_) + , check_keywords_after_name(check_keywords_after_name_) { } @@ -104,6 +106,7 @@ protected: bool require_type = true; bool allow_null_modifiers = false; + bool check_keywords_after_name = false; }; using ParserColumnDeclaration = IParserColumnDeclaration; @@ -122,6 +125,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ParserKeyword s_comment{"COMMENT"}; ParserKeyword s_codec{"CODEC"}; ParserKeyword s_ttl{"TTL"}; + ParserKeyword s_remove{"REMOVE"}; ParserTernaryOperatorExpression expr_parser; ParserStringLiteral string_literal_parser; ParserCodec codec_parser; @@ -132,6 +136,24 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E if (!name_parser.parse(pos, name, expected)) return false; + const auto column_declaration = std::make_shared(); + tryGetIdentifierNameInto(name, column_declaration->name); + + /// This keyword may occur only in MODIFY COLUMN query. We check it here + /// because ParserDataType parses types as an arbitrary identifiers and + /// doesn't check that parsed string is existing data type. In this way + /// REMOVE keyword can be parsed as data type and further parsing will fail. + /// So we just check this keyword and in case of success return column + /// column declaration with name only. + if (s_remove.checkWithoutMoving(pos, expected)) + { + if (!check_keywords_after_name) + return false; + + node = column_declaration; + return true; + } + /** column name should be followed by type name if it * is not immediately followed by {DEFAULT, MATERIALIZED, ALIAS, COMMENT} */ @@ -197,9 +219,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E return false; } - const auto column_declaration = std::make_shared(); node = column_declaration; - tryGetIdentifierNameInto(name, column_declaration->name); if (type) { diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index caf98e911ab..3b062c2d689 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -43,6 +43,30 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } +namespace +{ + +AlterCommand::RemoveProperty removePropertyFromString(const String & property) +{ + if (property.empty()) + return AlterCommand::RemoveProperty::NO_PROPERTY; + else if (property == "DEFAULT") + return AlterCommand::RemoveProperty::DEFAULT; + else if (property == "MATERIALIZED") + return AlterCommand::RemoveProperty::MATERIALIZED; + else if (property == "ALIAS") + return AlterCommand::RemoveProperty::ALIAS; + else if (property == "COMMENT") + return AlterCommand::RemoveProperty::COMMENT; + else if (property == "CODEC") + return AlterCommand::RemoveProperty::CODEC; + else if (property == "TTL") + return AlterCommand::RemoveProperty::TTL; + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot remove unknown property '{}'", property); +} + +} std::optional AlterCommand::parse(const ASTAlterCommand * command_ast) { @@ -111,8 +135,9 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.type = AlterCommand::MODIFY_COLUMN; const auto & ast_col_decl = command_ast->col_decl->as(); - command.column_name = ast_col_decl.name; + command.to_remove = removePropertyFromString(command_ast->remove_property); + if (ast_col_decl.type) { command.data_type = data_type_factory.get(ast_col_decl.type); @@ -237,6 +262,13 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.ttl = command_ast->ttl; return command; } + else if (command_ast->type == ASTAlterCommand::REMOVE_TTL) + { + AlterCommand command; + command.ast = command_ast->clone(); + command.type = AlterCommand::REMOVE_TTL; + return command; + } else if (command_ast->type == ASTAlterCommand::MODIFY_SETTING) { AlterCommand command; @@ -301,24 +333,45 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con { metadata.columns.modify(column_name, after_column, first, [&](ColumnDescription & column) { - if (codec) - column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type ? data_type : column.type, false); - - if (comment) - column.comment = *comment; - - if (ttl) - column.ttl = ttl; - - if (data_type) - column.type = data_type; - - /// User specified default expression or changed - /// datatype. We have to replace default. - if (default_expression || data_type) + if (to_remove == RemoveProperty::DEFAULT + || to_remove == RemoveProperty::MATERIALIZED + || to_remove == RemoveProperty::ALIAS) { - column.default_desc.kind = default_kind; - column.default_desc.expression = default_expression; + column.default_desc = ColumnDefault{}; + } + else if (to_remove == RemoveProperty::CODEC) + { + column.codec.reset(); + } + else if (to_remove == RemoveProperty::COMMENT) + { + column.comment = String{}; + } + else if (to_remove == RemoveProperty::TTL) + { + column.ttl.reset(); + } + else + { + if (codec) + column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type ? data_type : column.type, false); + + if (comment) + column.comment = *comment; + + if (ttl) + column.ttl = ttl; + + if (data_type) + column.type = data_type; + + /// User specified default expression or changed + /// datatype. We have to replace default. + if (default_expression || data_type) + { + column.default_desc.kind = default_kind; + column.default_desc.expression = default_expression; + } } }); @@ -450,6 +503,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con { metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST(ttl, metadata.columns, context, metadata.primary_key); } + else if (type == REMOVE_TTL) + { + metadata.table_ttl = TTLTableDescription{}; + } else if (type == MODIFY_QUERY) { metadata.select = SelectQueryDescription::getSelectQueryFromASTForMatView(select, context); @@ -584,6 +641,10 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada if (ignore) return false; + /// We remove properties on metadata level + if (isRemovingProperty() || type == REMOVE_TTL) + return false; + if (type == DROP_COLUMN || type == DROP_INDEX || type == RENAME_COLUMN) return true; @@ -636,6 +697,11 @@ bool AlterCommand::isTTLAlter(const StorageInMemoryMetadata & metadata) const return ttl_changed; } +bool AlterCommand::isRemovingProperty() const +{ + return to_remove != RemoveProperty::NO_PROPERTY; +} + std::optional AlterCommand::tryConvertToMutationCommand(StorageInMemoryMetadata & metadata, const Context & context) const { if (!isRequireMutationStage(metadata)) @@ -716,6 +782,8 @@ String alterTypeToString(const AlterCommand::Type type) return "MODIFY QUERY"; case AlterCommand::Type::RENAME_COLUMN: return "RENAME COLUMN"; + case AlterCommand::Type::REMOVE_TTL: + return "REMOVE TTL"; } __builtin_unreachable(); } @@ -783,14 +851,15 @@ void AlterCommands::prepare(const StorageInMemoryMetadata & metadata) if (!has_column && command.if_exists) command.ignore = true; - if (has_column && command.data_type) + if (has_column) { auto column_from_table = columns.get(command.column_name); - if (!command.default_expression && column_from_table.default_desc.expression) + if (command.data_type && !command.default_expression && column_from_table.default_desc.expression) { command.default_kind = column_from_table.default_desc.kind; command.default_expression = column_from_table.default_desc.expression; } + } } else if (command.type == AlterCommand::ADD_COLUMN) @@ -857,6 +926,70 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con if (command.codec) CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context.getSettingsRef().allow_suspicious_codecs); + auto column_default = all_columns.getDefault(column_name); + if (column_default) + { + if (command.to_remove == AlterCommand::RemoveProperty::DEFAULT && column_default->kind != ColumnDefaultKind::Default) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot remove DEFAULT from column {}, because column default type is {}. Use REMOVE {} to delete it", + backQuote(column_name), toString(column_default->kind), toString(column_default->kind)); + } + if (command.to_remove == AlterCommand::RemoveProperty::MATERIALIZED && column_default->kind != ColumnDefaultKind::Materialized) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot remove MATERIALIZED from column {}, because column default type is {}. Use REMOVE {} to delete it", + backQuote(column_name), toString(column_default->kind), toString(column_default->kind)); + } + if (command.to_remove == AlterCommand::RemoveProperty::ALIAS && column_default->kind != ColumnDefaultKind::Alias) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot remove ALIAS from column {}, because column default type is {}. Use REMOVE {} to delete it", + backQuote(column_name), toString(column_default->kind), toString(column_default->kind)); + } + } + + if (command.isRemovingProperty()) + { + if (!column_default && command.to_remove == AlterCommand::RemoveProperty::DEFAULT) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Column {} doesn't have DEFAULT, cannot remove it", + backQuote(column_name)); + + if (!column_default && command.to_remove == AlterCommand::RemoveProperty::ALIAS) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Column {} doesn't have ALIAS, cannot remove it", + backQuote(column_name)); + + if (!column_default && command.to_remove == AlterCommand::RemoveProperty::MATERIALIZED) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Column {} doesn't have MATERIALIZED, cannot remove it", + backQuote(column_name)); + + auto column_from_table = all_columns.get(column_name); + if (command.to_remove == AlterCommand::RemoveProperty::TTL && column_from_table.ttl == nullptr) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Column {} doesn't have TTL, cannot remove it", + backQuote(column_name)); + if (command.to_remove == AlterCommand::RemoveProperty::CODEC && column_from_table.codec == nullptr) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Column {} doesn't have TTL, cannot remove it", + backQuote(column_name)); + if (command.to_remove == AlterCommand::RemoveProperty::COMMENT && column_from_table.comment.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Column {} doesn't have COMMENT, cannot remove it", + backQuote(column_name)); + + } modified_columns.emplace(column_name); } @@ -966,6 +1099,10 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con throw Exception{"Cannot rename column from nested struct to normal column and vice versa", ErrorCodes::BAD_ARGUMENTS}; } } + else if (command.type == AlterCommand::REMOVE_TTL && !metadata.hasAnyTableTTL()) + { + throw Exception{"Table doesn't have any table TTL expression, cannot remove", ErrorCodes::BAD_ARGUMENTS}; + } /// Collect default expressions for MODIFY and ADD comands if (command.type == AlterCommand::MODIFY_COLUMN || command.type == AlterCommand::ADD_COLUMN) diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index 3578507a361..c973b0b6a6f 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -37,6 +37,22 @@ struct AlterCommand MODIFY_SETTING, MODIFY_QUERY, RENAME_COLUMN, + REMOVE_TTL, + }; + + /// Which property user wants to remove from column + enum class RemoveProperty + { + NO_PROPERTY, + /// Default specifiers + DEFAULT, + MATERIALIZED, + ALIAS, + + /// Other properties + COMMENT, + CODEC, + TTL }; Type type; @@ -107,16 +123,13 @@ struct AlterCommand /// Target column name String rename_to; + /// What to remove from column (or TTL) + RemoveProperty to_remove = RemoveProperty::NO_PROPERTY; + static std::optional parse(const ASTAlterCommand * command); void apply(StorageInMemoryMetadata & metadata, const Context & context) const; - /// Checks that alter query changes data. For MergeTree: - /// * column files (data and marks) - /// * each part meta (columns.txt) - /// in each part on disk (it's not lightweight alter). - bool isModifyingData(const StorageInMemoryMetadata & metadata) const; - /// Check that alter command require data modification (mutation) to be /// executed. For example, cast from Date to UInt16 type can be executed /// without any data modifications. But column drop or modify from UInt16 to @@ -132,6 +145,9 @@ struct AlterCommand /// Checks that any TTL changed by alter bool isTTLAlter(const StorageInMemoryMetadata & metadata) const; + /// Command removing some property from column or table + bool isRemovingProperty() const; + /// If possible, convert alter command to mutation command. In other case /// return empty optional. Some storages may execute mutations after /// metadata changes. @@ -164,9 +180,6 @@ public: /// Commands have to be prepared before apply. void apply(StorageInMemoryMetadata & metadata, const Context & context) const; - /// At least one command modify data on disk. - bool isModifyingData(const StorageInMemoryMetadata & metadata) const; - /// At least one command modify settings. bool isSettingsAlter() const; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6258ffd40fe..65c0c5ac313 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -900,10 +900,17 @@ ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & m if (metadata_diff.ttl_table_changed) { - ParserTTLExpressionList parser; - auto ttl_for_table_ast = parseQuery(parser, metadata_diff.new_ttl_table, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - new_metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( - ttl_for_table_ast, new_metadata.columns, global_context, new_metadata.primary_key); + if (!metadata_diff.new_ttl_table.empty()) + { + ParserTTLExpressionList parser; + auto ttl_for_table_ast = parseQuery(parser, metadata_diff.new_ttl_table, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); + new_metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( + ttl_for_table_ast, new_metadata.columns, global_context, new_metadata.primary_key); + } + else /// TTL was removed + { + new_metadata.table_ttl = TTLTableDescription{}; + } } } @@ -3818,7 +3825,12 @@ void StorageReplicatedMergeTree::alter( future_metadata_in_zk.partition_key = serializeAST(*future_metadata.partition_key.expression_list_ast); if (ast_to_str(future_metadata.table_ttl.definition_ast) != ast_to_str(current_metadata->table_ttl.definition_ast)) - future_metadata_in_zk.ttl_table = serializeAST(*future_metadata.table_ttl.definition_ast); + { + if (future_metadata.table_ttl.definition_ast) + future_metadata_in_zk.ttl_table = serializeAST(*future_metadata.table_ttl.definition_ast); + else /// TTL was removed + future_metadata_in_zk.ttl_table = ""; + } String new_indices_str = future_metadata.secondary_indices.toString(); if (new_indices_str != current_metadata->secondary_indices.toString()) diff --git a/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.reference b/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.reference new file mode 100644 index 00000000000..e7e7c2e6ad1 --- /dev/null +++ b/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.reference @@ -0,0 +1,4 @@ +CREATE TABLE default.no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = MergeTree()\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = MergeTree()\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/01493_r_no_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/01493_r_no_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.sql b/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.sql new file mode 100644 index 00000000000..aceb4e8140d --- /dev/null +++ b/tests/queries/0_stateless/01493_alter_remove_no_property_zookeeper.sql @@ -0,0 +1,51 @@ +DROP TABLE IF EXISTS no_prop_table; + +CREATE TABLE no_prop_table +( + some_column UInt64 +) +ENGINE MergeTree() +ORDER BY tuple(); + +SHOW CREATE TABLE no_prop_table; + +-- just nothing happened +ALTER TABLE no_prop_table MODIFY COLUMN some_column REMOVE DEFAULT; --{serverError 36} +ALTER TABLE no_prop_table MODIFY COLUMN some_column REMOVE MATERIALIZED; --{serverError 36} +ALTER TABLE no_prop_table MODIFY COLUMN some_column REMOVE ALIAS; --{serverError 36} +ALTER TABLE no_prop_table MODIFY COLUMN some_column REMOVE CODEC; --{serverError 36} +ALTER TABLE no_prop_table MODIFY COLUMN some_column REMOVE COMMENT; --{serverError 36} +ALTER TABLE no_prop_table MODIFY COLUMN some_column REMOVE TTL; --{serverError 36} + +ALTER TABLE no_prop_table REMOVE TTL; --{serverError 36} + +SHOW CREATE TABLE no_prop_table; + +DROP TABLE IF EXISTS no_prop_table; + +DROP TABLE IF EXISTS r_no_prop_table; + +CREATE TABLE r_no_prop_table +( + some_column UInt64 +) +ENGINE ReplicatedMergeTree('/clickhouse/test/01493_r_no_prop_table', '1') +ORDER BY tuple(); + +SHOW CREATE TABLE r_no_prop_table; + +ALTER TABLE r_no_prop_table MODIFY COLUMN some_column REMOVE DEFAULT; --{serverError 36} +ALTER TABLE r_no_prop_table MODIFY COLUMN some_column REMOVE MATERIALIZED; --{serverError 36} +ALTER TABLE r_no_prop_table MODIFY COLUMN some_column REMOVE ALIAS; --{serverError 36} +ALTER TABLE r_no_prop_table MODIFY COLUMN some_column REMOVE CODEC; --{serverError 36} +ALTER TABLE r_no_prop_table MODIFY COLUMN some_column REMOVE COMMENT; --{serverError 36} +ALTER TABLE r_no_prop_table MODIFY COLUMN some_column REMOVE TTL; --{serverError 36} + +ALTER TABLE r_no_prop_table REMOVE TTL; --{serverError 36} + +SHOW CREATE TABLE r_no_prop_table; + +ALTER TABLE r_no_prop_table MODIFY COLUMN some_column REMOVE ttl; --{serverError 36} +ALTER TABLE r_no_prop_table remove TTL; --{serverError 36} + +DROP TABLE IF EXISTS r_no_prop_table; diff --git a/tests/queries/0_stateless/01493_alter_remove_properties.reference b/tests/queries/0_stateless/01493_alter_remove_properties.reference new file mode 100644 index 00000000000..57b0ff22a4b --- /dev/null +++ b/tests/queries/0_stateless/01493_alter_remove_properties.reference @@ -0,0 +1,20 @@ +CREATE TABLE default.prop_table\n(\n `column_default` UInt64 DEFAULT 42,\n `column_materialized` UInt64 MATERIALIZED column_default * 42,\n `column_alias` UInt64 ALIAS column_default + 1,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date COMMENT \'Some comment\',\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = MergeTree()\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +42 1764 43 str 2019-10-01 1 +CREATE TABLE default.prop_table\n(\n `column_default` UInt64 DEFAULT 42,\n `column_materialized` UInt64 MATERIALIZED column_default * 42,\n `column_alias` UInt64 ALIAS column_default + 1,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = MergeTree()\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.prop_table\n(\n `column_default` UInt64 DEFAULT 42,\n `column_materialized` UInt64 MATERIALIZED column_default * 42,\n `column_alias` UInt64 ALIAS column_default + 1,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = MergeTree()\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +42 1764 0 str 2019-10-01 1 +CREATE TABLE default.prop_table\n(\n `column_default` UInt64 DEFAULT 42,\n `column_materialized` UInt64 MATERIALIZED column_default * 42,\n `column_alias` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = MergeTree()\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +42 1764 0 str 2019-10-01 1 +42 1764 33 trs 2020-01-01 2 +CREATE TABLE default.prop_table\n(\n `column_default` UInt64 DEFAULT 42,\n `column_materialized` UInt64,\n `column_alias` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = MergeTree()\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +42 1764 0 str 2019-10-01 1 +42 1764 33 trs 2020-01-01 2 +42 11 44 rts 2020-02-01 3 +CREATE TABLE default.prop_table\n(\n `column_default` UInt64,\n `column_materialized` UInt64,\n `column_alias` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = MergeTree()\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +42 1764 0 str 2019-10-01 1 +42 1764 33 trs 2020-01-01 2 +42 11 44 rts 2020-02-01 3 +0 22 55 tsr 2020-03-01 4 +CREATE TABLE default.prop_table\n(\n `column_default` UInt64,\n `column_materialized` UInt64,\n `column_alias` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = MergeTree()\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.prop_table\n(\n `column_default` UInt64,\n `column_materialized` UInt64,\n `column_alias` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = MergeTree()\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +4 diff --git a/tests/queries/0_stateless/01493_alter_remove_properties.sql b/tests/queries/0_stateless/01493_alter_remove_properties.sql new file mode 100644 index 00000000000..01213ccd826 --- /dev/null +++ b/tests/queries/0_stateless/01493_alter_remove_properties.sql @@ -0,0 +1,72 @@ +DROP TABLE IF EXISTS prop_table; + +CREATE TABLE prop_table +( + column_default UInt64 DEFAULT 42, + column_materialized UInt64 MATERIALIZED column_default * 42, + column_alias UInt64 ALIAS column_default + 1, + column_codec String CODEC(ZSTD(10)), + column_comment Date COMMENT 'Some comment', + column_ttl UInt64 TTL column_comment + INTERVAL 1 MONTH +) +ENGINE MergeTree() +ORDER BY tuple() +TTL column_comment + INTERVAL 2 MONTH; + +SHOW CREATE TABLE prop_table; + +SYSTEM STOP TTL MERGES prop_table; + +INSERT INTO prop_table (column_codec, column_comment, column_ttl) VALUES ('str', toDate('2019-10-01'), 1); + +SELECT column_default, column_materialized, column_alias, column_codec, column_comment, column_ttl FROM prop_table; + +ALTER TABLE prop_table MODIFY COLUMN column_comment REMOVE COMMENT; + +SHOW CREATE TABLE prop_table; + +ALTER TABLE prop_table MODIFY COLUMN column_codec REMOVE CODEC; + +SHOW CREATE TABLE prop_table; + +ALTER TABLE prop_table MODIFY COLUMN column_alias REMOVE ALIAS; + +SELECT column_default, column_materialized, column_alias, column_codec, column_comment, column_ttl FROM prop_table; + +SHOW CREATE TABLE prop_table; + +INSERT INTO prop_table (column_alias, column_codec, column_comment, column_ttl) VALUES (33, 'trs', toDate('2020-01-01'), 2); + +SELECT column_default, column_materialized, column_alias, column_codec, column_comment, column_ttl FROM prop_table ORDER BY column_ttl; + +ALTER TABLE prop_table MODIFY COLUMN column_materialized REMOVE MATERIALIZED; + +SHOW CREATE TABLE prop_table; + +INSERT INTO prop_table (column_materialized, column_alias, column_codec, column_comment, column_ttl) VALUES (11, 44, 'rts', toDate('2020-02-01'), 3); + +SELECT column_default, column_materialized, column_alias, column_codec, column_comment, column_ttl FROM prop_table ORDER BY column_ttl; + +ALTER TABLE prop_table MODIFY COLUMN column_default REMOVE DEFAULT; + +SHOW CREATE TABLE prop_table; + +INSERT INTO prop_table (column_materialized, column_alias, column_codec, column_comment, column_ttl) VALUES (22, 55, 'tsr', toDate('2020-03-01'), 4); + +SELECT column_default, column_materialized, column_alias, column_codec, column_comment, column_ttl FROM prop_table ORDER BY column_ttl; + +ALTER TABLE prop_table REMOVE TTL; + +SHOW CREATE TABLE prop_table; + +ALTER TABLE prop_table MODIFY COLUMN column_ttl REMOVE TTL; + +SHOW CREATE TABLE prop_table; + +SYSTEM START TTL MERGES prop_table; + +OPTIMIZE TABLE prop_table FINAL; + +SELECT COUNT() FROM prop_table; + +DROP TABLE IF EXISTS prop_table; diff --git a/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.reference b/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.reference new file mode 100644 index 00000000000..7cd2c81b460 --- /dev/null +++ b/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.reference @@ -0,0 +1,21 @@ +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date COMMENT \'Some comment\',\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date COMMENT \'Some comment\',\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +====== remove column comment ====== +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +====== remove column codec ====== +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +====== remove column default ====== +42 str 1 +0 tsr 2 +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +====== remove column TTL ====== +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192 +====== remove table TTL ====== +CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +3 +3 diff --git a/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql b/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql new file mode 100644 index 00000000000..c095c4216fd --- /dev/null +++ b/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql @@ -0,0 +1,92 @@ +DROP TABLE IF EXISTS r_prop_table1; +DROP TABLE IF EXISTS r_prop_table2; + +SET replication_alter_partitions_sync = 2; + +CREATE TABLE r_prop_table1 +( + column_default UInt64 DEFAULT 42, + column_codec String CODEC(ZSTD(10)), + column_comment Date COMMENT 'Some comment', + column_ttl UInt64 TTL column_comment + INTERVAL 1 MONTH +) +ENGINE ReplicatedMergeTree('/clickhouse/test_01493/r_prop_table', '1') +ORDER BY tuple() +TTL column_comment + INTERVAL 2 MONTH; + +CREATE TABLE r_prop_table2 +( + column_default UInt64 DEFAULT 42, + column_codec String CODEC(ZSTD(10)), + column_comment Date COMMENT 'Some comment', + column_ttl UInt64 TTL column_comment + INTERVAL 1 MONTH +) +ENGINE ReplicatedMergeTree('/clickhouse/test_01493/r_prop_table', '2') +ORDER BY tuple() +TTL column_comment + INTERVAL 2 MONTH; + +SHOW CREATE TABLE r_prop_table1; +SHOW CREATE TABLE r_prop_table2; + +INSERT INTO r_prop_table1 (column_codec, column_comment, column_ttl) VALUES ('str', toDate('2020-10-01'), 1); + +SYSTEM SYNC REPLICA r_prop_table2; + +SELECT '====== remove column comment ======'; +ALTER TABLE r_prop_table1 MODIFY COLUMN column_comment REMOVE COMMENT; + +SHOW CREATE TABLE r_prop_table1; +SHOW CREATE TABLE r_prop_table2; + +DETACH TABLE r_prop_table1; +ATTACH TABLE r_prop_table1; + +SELECT '====== remove column codec ======'; +ALTER TABLE r_prop_table2 MODIFY COLUMN column_codec REMOVE CODEC; + +SHOW CREATE TABLE r_prop_table1; +SHOW CREATE TABLE r_prop_table2; + +SELECT '====== remove column default ======'; +ALTER TABLE r_prop_table2 MODIFY COLUMN column_default REMOVE DEFAULT; + +INSERT INTO r_prop_table1 (column_codec, column_comment, column_ttl) VALUES ('tsr', now(), 2); + +SYSTEM SYNC REPLICA r_prop_table2; + +SELECT column_default, column_codec, column_ttl FROM r_prop_table1 ORDER BY column_ttl; + +DETACH TABLE r_prop_table2; +ATTACH TABLE r_prop_table2; + +SHOW CREATE TABLE r_prop_table1; +SHOW CREATE TABLE r_prop_table2; + +SELECT '====== remove column TTL ======'; +ALTER TABLE r_prop_table2 MODIFY COLUMN column_ttl REMOVE TTL; + +SHOW CREATE TABLE r_prop_table1; +SHOW CREATE TABLE r_prop_table2; + +SELECT '====== remove table TTL ======'; +ALTER TABLE r_prop_table1 REMOVE TTL; + +INSERT INTO r_prop_table1 (column_codec, column_comment, column_ttl) VALUES ('rts', now() - INTERVAL 1 YEAR, 3); + +SYSTEM SYNC REPLICA r_prop_table2; + +DETACH TABLE r_prop_table2; +ATTACH TABLE r_prop_table2; + +SHOW CREATE TABLE r_prop_table1; +SHOW CREATE TABLE r_prop_table2; + +OPTIMIZE TABLE r_prop_table2 FINAL; + +SYSTEM SYNC REPLICA r_prop_table1; + +SELECT COUNT() FROM r_prop_table1; +SELECT COUNT() FROM r_prop_table2; + +DROP TABLE IF EXISTS r_prop_table1; +DROP TABLE IF EXISTS r_prop_table2; diff --git a/tests/queries/0_stateless/01493_alter_remove_wrong_default.reference b/tests/queries/0_stateless/01493_alter_remove_wrong_default.reference new file mode 100644 index 00000000000..27120d582e4 --- /dev/null +++ b/tests/queries/0_stateless/01493_alter_remove_wrong_default.reference @@ -0,0 +1 @@ +CREATE TABLE default.default_table\n(\n `key` UInt64 DEFAULT 42,\n `value1` UInt64 MATERIALIZED key * key,\n `value2` UInt64 ALIAS value1 * key\n)\nENGINE = MergeTree()\nORDER BY tuple()\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01493_alter_remove_wrong_default.sql b/tests/queries/0_stateless/01493_alter_remove_wrong_default.sql new file mode 100644 index 00000000000..2099604ec13 --- /dev/null +++ b/tests/queries/0_stateless/01493_alter_remove_wrong_default.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS default_table; + +CREATE TABLE default_table ( + key UInt64 DEFAULT 42, + value1 UInt64 MATERIALIZED key * key, + value2 ALIAS value1 * key +) +ENGINE = MergeTree() +ORDER BY tuple(); + +ALTER TABLE default_table MODIFY COLUMN key REMOVE MATERIALIZED; --{serverError 36} +ALTER TABLE default_table MODIFY COLUMN key REMOVE ALIAS; --{serverError 36} + +ALTER TABLE default_table MODIFY COLUMN value1 REMOVE DEFAULT; --{serverError 36} +ALTER TABLE default_table MODIFY COLUMN value1 REMOVE ALIAS; --{serverError 36} + +ALTER TABLE default_table MODIFY COLUMN value2 REMOVE DEFAULT; --{serverError 36} +ALTER TABLE default_table MODIFY COLUMN value2 REMOVE MATERIALIZED; --{serverError 36} + +SHOW CREATE TABLE default_table; + +DROP TABLE IF EXISTS default_table; From a4337635ccb87cbcc5585672f2c9d5c9a8e218b0 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 20 Sep 2020 19:42:11 +0300 Subject: [PATCH 228/273] Update single_fixed_string_groupby.xml --- tests/performance/single_fixed_string_groupby.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/performance/single_fixed_string_groupby.xml b/tests/performance/single_fixed_string_groupby.xml index 10ca5f1ebcd..013de70c11a 100644 --- a/tests/performance/single_fixed_string_groupby.xml +++ b/tests/performance/single_fixed_string_groupby.xml @@ -10,8 +10,8 @@ INSERT INTO perf_lc_fixed_str_groupby SELECT ('number key ' || toString(number % 400)) AS a, ('number key ' || toString(number % 20)) AS b FROM numbers(30000000) - SELECT count() FROM perf_lc_fixed_str_groupby GROUP BY a - SELECT count() FROM perf_lc_fixed_str_groupby GROUP BY b + SELECT count() FROM perf_lc_fixed_str_groupby GROUP BY a + SELECT count() FROM perf_lc_fixed_str_groupby GROUP BY b DROP TABLE IF EXISTS perf_lc_fixed_str_groupby From 5244653493f963eff2582726536ca6fd55093d14 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 20 Sep 2020 19:45:27 +0300 Subject: [PATCH 229/273] Update maxmap.md --- docs/ru/sql-reference/aggregate-functions/reference/maxmap.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/maxmap.md b/docs/ru/sql-reference/aggregate-functions/reference/maxmap.md index efac771666b..af817ee1d04 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/maxmap.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/maxmap.md @@ -6,13 +6,13 @@ toc_priority: 143 Синтаксис: `maxMap(key, value)` or `maxMap(Tuple(key, value))` -Вычисляет максимальное значение массива `value` в соответствии с ключами, указанными в массиве `key`. +Вычисляет максимальные значения массива `value`, соответствующие ключам, указанным в массиве `key`. Передача кортежа ключей и массивов значений идентична передаче двух массивов ключей и значений. Количество элементов в параметрах `key` и `value` должно быть одинаковым для каждой суммируемой строки. -Возвращает кортеж из двух массивов: ключи в отсортированном порядке и значения, рассчитанные для соответствующих ключей. +Возвращает кортеж из двух массивов: ключи и значения, рассчитанные для соответствующих ключей. Пример: From 757be58fca4dd771231ee423d5f6ed1db8e90d5b Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 20 Sep 2020 19:45:43 +0300 Subject: [PATCH 230/273] Update maxmap.md --- docs/en/sql-reference/aggregate-functions/reference/maxmap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/maxmap.md b/docs/en/sql-reference/aggregate-functions/reference/maxmap.md index ffb36f06e61..46f0881ba49 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/maxmap.md +++ b/docs/en/sql-reference/aggregate-functions/reference/maxmap.md @@ -12,7 +12,7 @@ Passing a tuple of keys and value ​​arrays is identical to passing two array The number of elements in `key` and `value` must be the same for each row that is totaled. -Returns a tuple of two arrays: keys in sorted order, and values calculated for the corresponding keys. +Returns a tuple of two arrays: keys and values calculated for the corresponding keys. Example: From b2e9d0f20f9050213170451127262c36bccfddd2 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 20 Sep 2020 19:45:57 +0300 Subject: [PATCH 231/273] Update maxmap.md --- docs/en/sql-reference/aggregate-functions/reference/maxmap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/maxmap.md b/docs/en/sql-reference/aggregate-functions/reference/maxmap.md index 46f0881ba49..c62502cf46e 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/maxmap.md +++ b/docs/en/sql-reference/aggregate-functions/reference/maxmap.md @@ -8,7 +8,7 @@ Syntax: `maxMap(key, value)` or `maxMap(Tuple(key, value))` Calculates the maximum from `value` array according to the keys specified in the `key` array. -Passing a tuple of keys and value ​​arrays is identical to passing two arrays of keys and values. +Passing a tuple of keys and value arrays is identical to passing two arrays of keys and values. The number of elements in `key` and `value` must be the same for each row that is totaled. From 92756f4635be4170969938e4be41fd20f1ec568f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 20 Sep 2020 19:46:44 +0300 Subject: [PATCH 232/273] Update type-conversion-functions.md --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 7a57b94c4cd..987f9f6ae85 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -515,7 +515,7 @@ SELECT parseDateTimeBestEffort('10 20:19') ## parseDateTimeBestEffortUS {#parsedatetimebesteffortUS} -Эта функция похожа на [‘parseDateTimeBestEffort’](#parsedatetimebesteffort), но разница состоит в том, что в она использует американский формат даты (`MM/DD/YYYY` etc.) в случае многозначности. +Эта функция похожа на [‘parseDateTimeBestEffort’](#parsedatetimebesteffort), но разница состоит в том, что в она предполагает американский формат даты (`MM/DD/YYYY` etc.) в случае неоднозначности. **Синтаксис** From 07f95d03994682fba72336695238a6d38c7b523a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 20 Sep 2020 23:07:17 +0300 Subject: [PATCH 233/273] Update lowcardinality.md --- docs/ru/sql-reference/data-types/lowcardinality.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/sql-reference/data-types/lowcardinality.md b/docs/ru/sql-reference/data-types/lowcardinality.md index fc10624600a..ec9e4e7588e 100644 --- a/docs/ru/sql-reference/data-types/lowcardinality.md +++ b/docs/ru/sql-reference/data-types/lowcardinality.md @@ -19,11 +19,11 @@ LowCardinality(data_type) ## Описание {#lowcardinality-dscr} -`LowCardinality` — это надстройка, изменяющая способ хранения и правила обработки данных. ClickHouse применяет [словарное кодирование](https://en.wikipedia.org/wiki/Dictionary_coder) в столбцы типа `LowCardinality`. Работа с данными, представленными в словарном виде, значительно увеличивает производительность запросов [SELECT](../statements/select/index.md) для многих приложений. +`LowCardinality` — это надстройка, изменяющая способ хранения и правила обработки данных. ClickHouse применяет [словарное кодирование](https://en.wikipedia.org/wiki/Dictionary_coder) в столбцы типа `LowCardinality`. Работа с данными, представленными в словарном виде, может значительно увеличивать производительность запросов [SELECT](../statements/select/index.md) для многих приложений. Эффективность использования типа данных `LowCarditality` зависит от разнообразия данных. Если словарь содержит менее 10 000 различных значений, ClickHouse в основном показывает более высокую эффективность чтения и хранения данных. Если же словарь содержит более 100 000 различных значений, ClickHouse может работать хуже, чем при использовании обычных типов данных. -При работе со строками используйте `LowCardinality` вместо [Enum](enum.md). `LowCardinality` обеспечивает большую гибкость в использовании и часто показывает такую же или более высокую эффективность. +При работе со строками, использование `LowCardinality` вместо [Enum](enum.md). `LowCardinality` обеспечивает большую гибкость в использовании и часто показывает такую же или более высокую эффективность. ## Пример @@ -56,4 +56,4 @@ ORDER BY id - [A Magical Mystery Tour of the LowCardinality Data Type](https://www.altinity.com/blog/2019/3/27/low-cardinality). - [Reducing Clickhouse Storage Cost with the Low Cardinality Type – Lessons from an Instana Engineer](https://www.instana.com/blog/reducing-clickhouse-storage-cost-with-the-low-cardinality-type-lessons-from-an-instana-engineer/). -- [String Optimization (video presentation in Russian)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [Slides in English](https://github.com/yandex/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf). \ No newline at end of file +- [String Optimization (video presentation in Russian)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [Slides in English](https://github.com/yandex/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf). From 5301de7656b4b0b92a8c28ecca119ce4e1935a75 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Sep 2020 23:43:28 +0300 Subject: [PATCH 234/273] Control heavy translation units --- CMakeLists.txt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index a879ad9b076..14f1fcb4a64 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -80,6 +80,11 @@ endif () include (cmake/find/ccache.cmake) +option(ENABLE_CHECK_HEAVY_BUILDS "Don't allow C++ translation units to compile too long or to take too much memory while compiling" OFF) +if (ENABLE_CHECK_HEAVY_BUILDS) + set (CMAKE_CXX_COMPILER_LAUNCHER prlimit --rss=10000000 --cpu=600) +endif () + if (NOT CMAKE_BUILD_TYPE OR CMAKE_BUILD_TYPE STREQUAL "None") set (CMAKE_BUILD_TYPE "RelWithDebInfo") message (STATUS "CMAKE_BUILD_TYPE is not set, set to default = ${CMAKE_BUILD_TYPE}") From 30f96dc525a6ba3540322346df4a0b790aed5634 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 20 Sep 2020 23:41:36 +0300 Subject: [PATCH 235/273] Fix possible UAF from jobs in the ThreadPool on shutdown ThreadPoolImpl<>::worker signaling that job is done while still storing std::function<> object, and this can lead to problems on shutdown, since in in this cast std::function<> can refers to some global/static object that had been already destroyed (typical example is Logger). I believe that this is exactly what TSAN reports about (decoded manually, since llvm-symbolizer does not work in the test env):
``` - 2020-09-20 17:44:43 Write of size 8 at 0x7b1000008f78 by main thread (mutexes: write M1432): operator delete(void*, unsigned long) ??:0:0 Poco::Logger::~Logger() /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Logger.cpp:39:1 non-virtual thunk to Poco::Logger::~Logger() /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Logger.cpp:0:0 Poco::RefCountedObject::release() const /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/include/Poco/RefCountedObject.h:82:24 Poco::Logger::shutdown() /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Logger.cpp:346:16 Poco::AutoLoggerShutdown::~AutoLoggerShutdown() /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Logger.cpp:459:4 cxa_at_exit_wrapper(void*) ??:0:0 ``` ``` - 2020-09-20 17:44:43 Previous atomic read of size 4 at 0x7b1000008f78 by thread T116: __tsan_atomic32_load ??:0:0 int std::__1::__cxx_atomic_load(std::__1::__cxx_atomic_base_impl const*, std::__1::memory_order) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/atomic:970:12 std::__1::__atomic_base::load(std::__1::memory_order) const /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/atomic:1487:17 std::__1::__atomic_base::operator int() const /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/atomic:1491:53 Poco::Logger::is(int) const /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/include/Poco/Logger.h:1264:9 MemoryTracker::logPeakMemoryUsage() const /build/obj-x86_64-linux-gnu/../src/Common/MemoryTracker.cpp:59:5 MemoryTracker::~MemoryTracker() /build/obj-x86_64-linux-gnu/../src/Common/MemoryTracker.cpp:46:13 DB::ThreadGroupStatus::~ThreadGroupStatus() /build/obj-x86_64-linux-gnu/../src/Common/ThreadStatus.h:51:7 std::__1::__shared_ptr_emplace >::__on_zero_shared() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3602:23 std::__1::__shared_count::__release_shared() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3440:9 std::__1::__shared_weak_count::__release_shared() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3482:27 std::__1::shared_ptr::~shared_ptr() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:4207:19 DB::PipelineExecutor::executeImpl(unsigned long)::$_4::~$_4() /build/obj-x86_64-linux-gnu/../src/Processors/Executors/PipelineExecutor.cpp:720:34 ThreadFromGlobalPool::ThreadFromGlobalPool(DB::PipelineExecutor::executeImpl(unsigned long)::$_4&&)::'lambda'()::~() /build/obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:161:54 std::__1::__compressed_pair_elem(DB::PipelineExecutor::executeImpl(unsigned long)::$_4&&)::'lambda'(), 0, false>::~__compressed_pair_elem() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2188:8 std::__1::__function::__alloc_func(DB::PipelineExecutor::executeImpl(unsigned long)::$_4&&)::'lambda'(), std::__1::allocator /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1559:38 std::__1::__function::__func(DB::PipelineExecutor::executeImpl(unsigned long)::$_4&&)::'lambda'(), std::__1::allocator /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1706:10 std::__1::__function::__value_func::~__value_func() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1828:19 std::__1::function::~function() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2460:43 ThreadPoolImpl::worker(std::__1::__list_iterator) /build/obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:268:5 void ThreadPoolImpl::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()::operator()() const /build/obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:116:73 decltype(std::__1::forward(fp)(std::__1::forward::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()>(fp0)...)) std::__1::__invoke::scheduleI> /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3519:1 void std::__1::__thread_execute >, void ThreadPoolImpl::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'(> /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:273:5 void* std::__1::__thread_proxy >, void ThreadPoolImpl::scheduleImpl(std::__1::function, int, std::__1::optional /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:284:5 __tsan_thread_start_func ??:0:0 ``` ``` - 2020-09-20 17:44:43 Mutex M1432 (0x0000181213a8) created at: pthread_mutex_init ??:0:0 Poco::MutexImpl::MutexImpl() /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Mutex_POSIX.cpp:64:6 Poco::Mutex::Mutex() /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Mutex.cpp:34:8 __cxx_global_var_init /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Logger.cpp:28:15 _GLOBAL__sub_I_Logger.cpp /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Logger.cpp:0:0 __libc_csu_init ??:0:0 ```
--- src/Common/ThreadPool.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 49516d777fb..f1fa82b7e68 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -233,6 +233,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ std::is_same_v ? CurrentMetrics::GlobalThreadActive : CurrentMetrics::LocalThreadActive); job(); + job = Job(); } catch (...) { From 035d7cb47f12bd8ffe7fa72af0d866e415752b83 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Sep 2020 23:46:32 +0300 Subject: [PATCH 236/273] Enable control for heavy translation units --- docker/packager/binary/build.sh | 2 +- release | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index dc34e7297dc..fd70b03242b 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -17,7 +17,7 @@ ccache --show-stats ||: ccache --zero-stats ||: ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so ||: rm -f CMakeCache.txt -cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=$BUILD_TYPE -DSANITIZE=$SANITIZER $CMAKE_FLAGS .. +cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=$BUILD_TYPE -DSANITIZE=$SANITIZER -DENABLE_CHECK_HEAVY_BUILDS=1 $CMAKE_FLAGS .. ninja $NINJA_FLAGS clickhouse-bundle mv ./programs/clickhouse* /output mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds diff --git a/release b/release index ec762b234fb..b20683a9caa 100755 --- a/release +++ b/release @@ -106,7 +106,7 @@ elif [[ $BUILD_TYPE == 'debug' ]]; then VERSION_POSTFIX+="+debug" fi -CMAKE_FLAGS=" $MALLOC_OPTS -DSANITIZE=$SANITIZER $CMAKE_FLAGS" +CMAKE_FLAGS=" $MALLOC_OPTS -DSANITIZE=$SANITIZER -DENABLE_CHECK_HEAVY_BUILDS=1 $CMAKE_FLAGS" [[ -n "$CMAKE_BUILD_TYPE" ]] && CMAKE_FLAGS=" -DCMAKE_BUILD_TYPE=$CMAKE_BUILD_TYPE $CMAKE_FLAGS" export CMAKE_FLAGS From 6e1bd79d51958603be38907c622493bcad9b0c8e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 21 Sep 2020 00:25:47 +0300 Subject: [PATCH 237/273] Add a test for #4476 --- .../0_stateless/01504_view_type_conversion.reference | 2 ++ .../0_stateless/01504_view_type_conversion.sql | 11 +++++++++++ 2 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/01504_view_type_conversion.reference create mode 100644 tests/queries/0_stateless/01504_view_type_conversion.sql diff --git a/tests/queries/0_stateless/01504_view_type_conversion.reference b/tests/queries/0_stateless/01504_view_type_conversion.reference new file mode 100644 index 00000000000..e32c81f4c4a --- /dev/null +++ b/tests/queries/0_stateless/01504_view_type_conversion.reference @@ -0,0 +1,2 @@ +[0,1,2,3,4,5,6,7,8,9] +['0','1','2','3','4','5','6','7','8','9'] diff --git a/tests/queries/0_stateless/01504_view_type_conversion.sql b/tests/queries/0_stateless/01504_view_type_conversion.sql new file mode 100644 index 00000000000..0133ecaf409 --- /dev/null +++ b/tests/queries/0_stateless/01504_view_type_conversion.sql @@ -0,0 +1,11 @@ +DROP TABLE IF EXISTS testv; + +create view testv(a UInt32) as select number a from numbers(10); +select groupArray(a) from testv; + +DROP TABLE testv; + +create view testv(a String) as select number a from numbers(10); +select groupArray(a) from testv; + +DROP TABLE testv; From fe0f8117afb5aa6c24538a202816c36b3072459a Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sun, 20 Sep 2020 19:22:57 -0300 Subject: [PATCH 238/273] more tests for CTE --- ...495_subqueries_in_with_statement.reference | 18 ++++++ .../01495_subqueries_in_with_statement.sql | 61 +++++++++++++++++++ 2 files changed, 79 insertions(+) diff --git a/tests/queries/0_stateless/01495_subqueries_in_with_statement.reference b/tests/queries/0_stateless/01495_subqueries_in_with_statement.reference index 8e851cd3ba1..16ca3450a74 100644 --- a/tests/queries/0_stateless/01495_subqueries_in_with_statement.reference +++ b/tests/queries/0_stateless/01495_subqueries_in_with_statement.reference @@ -12,3 +12,21 @@ 4 5 4 5 4 5 +--------------------------- +42 +42 +0 +42 +42 +42 +\N +42 +42 +42 +42 +42 +45 +\N +\N +42 +42 diff --git a/tests/queries/0_stateless/01495_subqueries_in_with_statement.sql b/tests/queries/0_stateless/01495_subqueries_in_with_statement.sql index 9ec921a9d4c..f34f4aaa6e0 100644 --- a/tests/queries/0_stateless/01495_subqueries_in_with_statement.sql +++ b/tests/queries/0_stateless/01495_subqueries_in_with_statement.sql @@ -11,3 +11,64 @@ SELECT * FROM (WITH test1 AS (SELECT toInt32(*) i FROM numbers(5)) SELECT * FROM WITH test1 AS (SELECT i + 1, j + 1 FROM test1) SELECT toInt64(4) i, toInt64(5) j FROM numbers(3) WHERE (i, j) IN test1; DROP TABLE IF EXISTS test1; + +select '---------------------------'; + +set empty_result_for_aggregation_by_empty_set = 0; + +WITH test1 AS (SELECT number-1 as n FROM numbers(42)) +SELECT max(n+1)+1 z FROM test1; + +WITH test1 AS (SELECT number-1 as n FROM numbers(42)) +SELECT max(n+1)+1 z FROM test1 join test1 x using n having z - 1 = (select min(n-1)+41 from test1) + 2; + +WITH test1 AS (SELECT number-1 as n FROM numbers(4442) limit 100) +SELECT max(n) FROM test1 where n=422; + +WITH test1 AS (SELECT number-1 as n FROM numbers(4442) limit 100) +SELECT max(n) FROM test1 where n=42; + +drop table if exists with_test ; +create table with_test engine=Memory as select cast(number-1 as Nullable(Int64)) n from numbers(10000); + +WITH test1 AS (SELECT n FROM with_test where n <= 40) +SELECT max(n+1)+1 z FROM test1 join test1 x using (n) having max(n+1)+1 - 1 = (select min(n-1)+41 from test1) + 2; + +WITH test1 AS (SELECT n FROM with_test where n <= 40) +SELECT max(n+1)+1 z FROM test1 join test1 x using (n) having z - 1 = (select min(n-1)+41 from test1) + 2; + +WITH test1 AS (SELECT n FROM with_test limit 100) +SELECT max(n) FROM test1 where n=422; + +WITH test1 AS (SELECT n FROM with_test limit 100) +SELECT max(n) FROM test1 where n=42; + +WITH test1 AS (SELECT n FROM with_test where n = 42 limit 100) +SELECT max(n) FROM test1 where n=42; + +WITH test1 AS (SELECT n FROM with_test where n = 42 or 1=1 limit 100) +SELECT max(n) FROM test1 where n=42; + +WITH test1 AS (SELECT n, null as b FROM with_test where n = 42 or b is null limit 100) +SELECT max(n) FROM test1 where n=42; + +WITH test1 AS (SELECT n, null b FROM with_test where b is null) +SELECT max(n) FROM test1 where n=42; + +WITH test1 AS (SELECT n, null b FROM with_test where b is null or 1=1) +SELECT max(n) FROM test1 where n=45; + +WITH test1 AS (SELECT n, null b FROM with_test where b is null and n = 42) +SELECT max(n) FROM test1 where n=45; + +WITH test1 AS (SELECT n, null b FROM with_test where 1=1 and n = 42) +SELECT max(n) FROM test1 where n=45; + +WITH test1 AS (SELECT n, null b, n+1 m FROM with_test where 1=0 or n = 42 limit 4) +SELECT max(n) m FROM test1 where test1.m=43 having max(n)=42; + +WITH test1 AS (SELECT n, null b, n+1 m FROM with_test where n = 42 limit 4) +SELECT max(n) m FROM test1 where b is null and test1.m=43 having m=42 limit 4; + +drop table with_test ; + From aa25df9d4538f87f56430f34da986e3f6e03cb88 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sun, 20 Sep 2020 19:43:19 -0300 Subject: [PATCH 239/273] order by added for quieries with limit --- .../01495_subqueries_in_with_statement.sql | 29 +++++++++---------- 1 file changed, 14 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/01495_subqueries_in_with_statement.sql b/tests/queries/0_stateless/01495_subqueries_in_with_statement.sql index f34f4aaa6e0..8102ed29fa8 100644 --- a/tests/queries/0_stateless/01495_subqueries_in_with_statement.sql +++ b/tests/queries/0_stateless/01495_subqueries_in_with_statement.sql @@ -16,40 +16,40 @@ select '---------------------------'; set empty_result_for_aggregation_by_empty_set = 0; -WITH test1 AS (SELECT number-1 as n FROM numbers(42)) +WITH test1 AS (SELECT number-1 as n FROM numbers(42)) SELECT max(n+1)+1 z FROM test1; -WITH test1 AS (SELECT number-1 as n FROM numbers(42)) +WITH test1 AS (SELECT number-1 as n FROM numbers(42)) SELECT max(n+1)+1 z FROM test1 join test1 x using n having z - 1 = (select min(n-1)+41 from test1) + 2; -WITH test1 AS (SELECT number-1 as n FROM numbers(4442) limit 100) +WITH test1 AS (SELECT number-1 as n FROM numbers(4442) order by n limit 100) SELECT max(n) FROM test1 where n=422; -WITH test1 AS (SELECT number-1 as n FROM numbers(4442) limit 100) +WITH test1 AS (SELECT number-1 as n FROM numbers(4442) order by n limit 100) SELECT max(n) FROM test1 where n=42; drop table if exists with_test ; create table with_test engine=Memory as select cast(number-1 as Nullable(Int64)) n from numbers(10000); -WITH test1 AS (SELECT n FROM with_test where n <= 40) +WITH test1 AS (SELECT n FROM with_test where n <= 40) SELECT max(n+1)+1 z FROM test1 join test1 x using (n) having max(n+1)+1 - 1 = (select min(n-1)+41 from test1) + 2; -WITH test1 AS (SELECT n FROM with_test where n <= 40) +WITH test1 AS (SELECT n FROM with_test where n <= 40) SELECT max(n+1)+1 z FROM test1 join test1 x using (n) having z - 1 = (select min(n-1)+41 from test1) + 2; -WITH test1 AS (SELECT n FROM with_test limit 100) +WITH test1 AS (SELECT n FROM with_test order by n limit 100) SELECT max(n) FROM test1 where n=422; -WITH test1 AS (SELECT n FROM with_test limit 100) +WITH test1 AS (SELECT n FROM with_test order by n limit 100) SELECT max(n) FROM test1 where n=42; -WITH test1 AS (SELECT n FROM with_test where n = 42 limit 100) +WITH test1 AS (SELECT n FROM with_test where n = 42 order by n limit 100) SELECT max(n) FROM test1 where n=42; -WITH test1 AS (SELECT n FROM with_test where n = 42 or 1=1 limit 100) +WITH test1 AS (SELECT n FROM with_test where n = 42 or 1=1 order by n limit 100) SELECT max(n) FROM test1 where n=42; -WITH test1 AS (SELECT n, null as b FROM with_test where n = 42 or b is null limit 100) +WITH test1 AS (SELECT n, null as b FROM with_test where n = 42 or b is null order by n limit 100) SELECT max(n) FROM test1 where n=42; WITH test1 AS (SELECT n, null b FROM with_test where b is null) @@ -61,14 +61,13 @@ SELECT max(n) FROM test1 where n=45; WITH test1 AS (SELECT n, null b FROM with_test where b is null and n = 42) SELECT max(n) FROM test1 where n=45; -WITH test1 AS (SELECT n, null b FROM with_test where 1=1 and n = 42) +WITH test1 AS (SELECT n, null b FROM with_test where 1=1 and n = 42 order by n) SELECT max(n) FROM test1 where n=45; -WITH test1 AS (SELECT n, null b, n+1 m FROM with_test where 1=0 or n = 42 limit 4) +WITH test1 AS (SELECT n, null b, n+1 m FROM with_test where 1=0 or n = 42 order by n limit 4) SELECT max(n) m FROM test1 where test1.m=43 having max(n)=42; -WITH test1 AS (SELECT n, null b, n+1 m FROM with_test where n = 42 limit 4) +WITH test1 AS (SELECT n, null b, n+1 m FROM with_test where n = 42 order by n limit 4) SELECT max(n) m FROM test1 where b is null and test1.m=43 having m=42 limit 4; drop table with_test ; - From 0119eb9c0dbafb26fe6acff06f03e1773e78f566 Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Mon, 21 Sep 2020 06:17:03 +0000 Subject: [PATCH 240/273] Bump mkdocs-macros-plugin from 0.4.9 to 0.4.13 in /docs/tools Bumps [mkdocs-macros-plugin](https://github.com/fralau/mkdocs_macros_plugin) from 0.4.9 to 0.4.13. - [Release notes](https://github.com/fralau/mkdocs_macros_plugin/releases) - [Commits](https://github.com/fralau/mkdocs_macros_plugin/commits) Signed-off-by: dependabot-preview[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index a3949892829..e31e43b99cd 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -18,7 +18,7 @@ Markdown==3.2.1 MarkupSafe==1.1.1 mkdocs==1.1.2 mkdocs-htmlproofer-plugin==0.0.3 -mkdocs-macros-plugin==0.4.9 +mkdocs-macros-plugin==0.4.13 nltk==3.5 nose==1.3.7 protobuf==3.13.0 From 27613f834681ef77b24e0e6f01e67a26017b61b0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 21 Sep 2020 10:38:11 +0300 Subject: [PATCH 241/273] Fix symbolizers path in test images --- docker/test/base/Dockerfile | 5 ++--- docker/test/integration/base/Dockerfile | 9 +-------- docker/test/stateless/run.sh | 7 ------- docker/test/stateless_unbundled/run.sh | 7 ------- docker/test/stress/run.sh | 2 -- docker/test/unit/Dockerfile | 9 +-------- docker/test/unit/perfraw/default.profraw | 0 7 files changed, 4 insertions(+), 35 deletions(-) create mode 100644 docker/test/unit/perfraw/default.profraw diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index 8117d2907bc..8adaf5ab543 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -1,7 +1,7 @@ # docker build -t yandex/clickhouse-test-base . -FROM ubuntu:19.10 +FROM ubuntu:20.04 -ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=10 +ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11 RUN apt-get update \ && apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \ @@ -43,7 +43,6 @@ RUN apt-get update \ llvm-${LLVM_VERSION} \ moreutils \ perl \ - perl \ pigz \ pkg-config \ tzdata \ diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 53627c78208..35decd907c0 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -1,5 +1,5 @@ # docker build -t yandex/clickhouse-integration-test . -FROM ubuntu:19.10 +FROM yandex/clickhouse-test-base RUN apt-get update \ && env DEBIAN_FRONTEND=noninteractive apt-get -y install \ @@ -8,7 +8,6 @@ RUN apt-get update \ libreadline-dev \ libicu-dev \ bsdutils \ - llvm-9 \ gdb \ unixodbc \ odbcinst \ @@ -29,9 +28,3 @@ RUN curl 'https://cdn.mysql.com//Downloads/Connector-ODBC/8.0/mysql-connector-od ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone - -# Sanitizer options -RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment; \ - echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ - echo "MSAN_OPTIONS='abort_on_error=1'" >> /etc/environment; \ - ln -s /usr/lib/llvm-9/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 4a9ad891883..b6b48cd0943 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -48,13 +48,6 @@ fi ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml -echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment -echo "TSAN_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment -echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment -echo "ASAN_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment -echo "UBSAN_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment -echo "LLVM_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment - service zookeeper start sleep 5 service clickhouse-server start && sleep 5 diff --git a/docker/test/stateless_unbundled/run.sh b/docker/test/stateless_unbundled/run.sh index 4a9ad891883..b6b48cd0943 100755 --- a/docker/test/stateless_unbundled/run.sh +++ b/docker/test/stateless_unbundled/run.sh @@ -48,13 +48,6 @@ fi ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml -echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment -echo "TSAN_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment -echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment -echo "ASAN_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment -echo "UBSAN_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment -echo "LLVM_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment - service zookeeper start sleep 5 service clickhouse-server start && sleep 5 diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 47c8603babb..8295e90b3ef 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -43,8 +43,6 @@ ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/u ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/ -echo "TSAN_OPTIONS='halt_on_error=1 history_size=7 ignore_noninstrumented_modules=1 verbosity=1'" >> /etc/environment -echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment start diff --git a/docker/test/unit/Dockerfile b/docker/test/unit/Dockerfile index ae5ea1820b0..0f65649fb76 100644 --- a/docker/test/unit/Dockerfile +++ b/docker/test/unit/Dockerfile @@ -5,12 +5,5 @@ ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone RUN apt-get install gdb -CMD ln -s /usr/lib/llvm-8/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; \ - echo "TSAN_OPTIONS='halt_on_error=1 history_size=7'" >> /etc/environment; \ - echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ - echo "ASAN_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment; \ - echo "UBSAN_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment; \ - echo "TSAN_SYMBOLIZER_PATH=/usr/lib/llvm-8/bin/llvm-symbolizer" >> /etc/environment; \ - echo "LLVM_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment; \ - service zookeeper start && sleep 7 && /usr/share/zookeeper/bin/zkCli.sh -server localhost:2181 -create create /clickhouse_test ''; \ +CMD service zookeeper start && sleep 7 && /usr/share/zookeeper/bin/zkCli.sh -server localhost:2181 -create create /clickhouse_test ''; \ gdb -q -ex 'set print inferior-events off' -ex 'set confirm off' -ex 'set print thread-events off' -ex run -ex bt -ex quit --args ./unit_tests_dbms | tee test_output/test_result.txt diff --git a/docker/test/unit/perfraw/default.profraw b/docker/test/unit/perfraw/default.profraw new file mode 100644 index 00000000000..e69de29bb2d From fd469f1266c61a85b5d898255243a1550249c4b6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 21 Sep 2020 10:41:23 +0300 Subject: [PATCH 242/273] Remove redundant file --- docker/test/unit/perfraw/default.profraw | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 docker/test/unit/perfraw/default.profraw diff --git a/docker/test/unit/perfraw/default.profraw b/docker/test/unit/perfraw/default.profraw deleted file mode 100644 index e69de29bb2d..00000000000 From aa3905989018bde484e78d060a792b8571c2ded6 Mon Sep 17 00:00:00 2001 From: Artemeey Date: Mon, 21 Sep 2020 11:10:27 +0300 Subject: [PATCH 243/273] Update storage_policies.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Описание move_factor --- docs/ru/operations/system-tables/storage_policies.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/system-tables/storage_policies.md b/docs/ru/operations/system-tables/storage_policies.md index f937654ab9a..df5c920b5ba 100644 --- a/docs/ru/operations/system-tables/storage_policies.md +++ b/docs/ru/operations/system-tables/storage_policies.md @@ -9,7 +9,7 @@ - `volume_priority` ([UInt64](../../sql-reference/data-types/int-uint.md)) — порядковый номер тома согласно конфигурации. - `disks` ([Array(String)](../../sql-reference/data-types/array.md)) — имена дисков, содержащихся в политике хранения. - `max_data_part_size` ([UInt64](../../sql-reference/data-types/int-uint.md)) — максимальный размер куска данных, который может храниться на дисках тома (0 — без ограничений). -- `move_factor` ([Float64](../../sql-reference/data-types/float.md))\` — доля свободного места, при превышении которой данные начинают перемещаться на следующий том. +- `move_factor` — доля доступного свободного места на томе, если места становится меньше, то данные начнут перемещение на следующий том, если он есть (по умолчанию 0.1). Если политика хранения содержит несколько томов, то каждому тому соответствует отдельная запись в таблице. From a7aa714a280b07f9074a33ee6b8301a094bb2f9e Mon Sep 17 00:00:00 2001 From: Artemeey Date: Mon, 21 Sep 2020 11:14:37 +0300 Subject: [PATCH 244/273] Update mergetree.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit описание move_factor --- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 881c11152cd..2f89317a0eb 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -563,7 +563,7 @@ ALTER TABLE example_table - `volume_name_N` — название тома. Названия томов должны быть уникальны. - `disk` — диск, находящийся внутри тома. - `max_data_part_size_bytes` — максимальный размер куска данных, который может находится на любом из дисков этого тома. -- `move_factor` — доля свободного места, при превышении которого данные начинают перемещаться на следующий том, если он есть (по умолчанию 0.1). +- `move_factor` — доля доступного свободного места на томе, если места становится меньше, то данные начнут перемещение на следующий том, если он есть (по умолчанию 0.1). Примеры конфигураций: From 54c1f04b3873726a48b3271c72f738ab384aa15b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 21 Sep 2020 11:36:12 +0300 Subject: [PATCH 245/273] Review fixes. --- src/Interpreters/InterpreterSelectQuery.cpp | 3 --- src/Processors/QueryPipeline.cpp | 2 +- src/Processors/QueryPipeline.h | 2 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 2 +- 4 files changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 33fa81383a3..5cdc9bfa5b5 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1899,9 +1899,6 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_p const Settings & settings = context->getSettingsRef(); - if (subqueries_for_sets.empty()) - return; - SizeLimits limits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode); addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), limits, *context); } diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index c8218023a57..4cbb4d9edb7 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -282,7 +282,7 @@ void QueryPipeline::addCreatingSetsTransform(const Block & res_header, SubqueryF pipe.addTransform(std::move(transform), totals_port, nullptr); } -void QueryPipeline::addDelayingPipeline(QueryPipeline pipeline) +void QueryPipeline::addPipelineBefore(QueryPipeline pipeline) { checkInitializedAndNotCompleted(); assertBlocksHaveEqualStructure(getHeader(), pipeline.getHeader(), "QueryPipeline"); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 2a318a53b76..80ae1d591a4 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -89,7 +89,7 @@ public: /// Add other pipeline and execute it before current one. /// Pipeline must have same header. - void addDelayingPipeline(QueryPipeline pipeline); + void addPipelineBefore(QueryPipeline pipeline); void addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, const Context & context); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 780e8493b40..5868a7045f7 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -95,7 +95,7 @@ QueryPipelinePtr CreatingSetsStep::updatePipeline(QueryPipelines pipelines) delayed_pipeline = std::move(*pipelines.front()); QueryPipelineProcessorsCollector collector(*main_pipeline, this); - main_pipeline->addDelayingPipeline(std::move(delayed_pipeline)); + main_pipeline->addPipelineBefore(std::move(delayed_pipeline)); auto added_processors = collector.detachProcessors(); processors.insert(processors.end(), added_processors.begin(), added_processors.end()); From 1bd76c820c035a7802b8448d0dd495469500e673 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 21 Sep 2020 12:05:10 +0300 Subject: [PATCH 246/273] Back to 19.10 --- docker/test/base/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index 8adaf5ab543..aa3f1d738c2 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -1,5 +1,5 @@ # docker build -t yandex/clickhouse-test-base . -FROM ubuntu:20.04 +FROM ubuntu:19.10 ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11 From ef85ce90c345fb593878ff1bd0c3e0729c729065 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 21 Sep 2020 13:17:33 +0300 Subject: [PATCH 247/273] fix build --- src/Common/renameat2.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Common/renameat2.cpp b/src/Common/renameat2.cpp index 5139a165c91..2eaae491aba 100644 --- a/src/Common/renameat2.cpp +++ b/src/Common/renameat2.cpp @@ -79,10 +79,9 @@ static bool renameat2(const std::string & old_path, const std::string & new_path #define RENAME_NOREPLACE -1 #define RENAME_EXCHANGE -1 -[[noreturn]] -static void renameat2(const std::string &, const std::string &, int) +static bool renameat2(const std::string &, const std::string &, int) { - throw Exception("Compiled without renameat2() support", ErrorCodes::UNSUPPORTED_METHOD); + return false } #endif From 450c1a5709c859285887ea7e370949b74b99f115 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 21 Sep 2020 13:19:02 +0300 Subject: [PATCH 248/273] Update renameat2.cpp --- src/Common/renameat2.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/renameat2.cpp b/src/Common/renameat2.cpp index 2eaae491aba..02acd8331d2 100644 --- a/src/Common/renameat2.cpp +++ b/src/Common/renameat2.cpp @@ -50,7 +50,7 @@ static bool supportsRenameat2Impl() static bool renameat2(const std::string & old_path, const std::string & new_path, int flags) { - if(!supportsRenameat2()) + if (!supportsRenameat2()) return false; if (old_path.empty() || new_path.empty()) throw Exception("Cannot rename " + old_path + " to " + new_path + ": path is empty", ErrorCodes::LOGICAL_ERROR); From efa493fbcb532772e46d2f43cfe8aee8931a9539 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 21 Sep 2020 13:29:31 +0300 Subject: [PATCH 249/273] Update SourceVisitor for GatherUtils. --- .../GatherUtils/ArraySourceVisitor.h | 10 +-- src/Functions/GatherUtils/Selectors.h | 76 +++++-------------- src/Functions/GatherUtils/concat.cpp | 61 +++++++++------ src/Functions/GatherUtils/has_all.cpp | 48 +++++++++++- src/Functions/GatherUtils/has_any.cpp | 48 +++++++++++- src/Functions/GatherUtils/has_substr.cpp | 48 +++++++++++- .../GatherUtils/sliceDynamicOffsetBounded.cpp | 32 +++++++- .../sliceDynamicOffsetUnbounded.cpp | 31 +++++++- .../sliceFromLeftConstantOffsetBounded.cpp | 31 +++++++- .../sliceFromLeftConstantOffsetUnbounded.cpp | 31 +++++++- .../sliceFromRightConstantOffsetBounded.cpp | 31 +++++++- .../sliceFromRightConstantOffsetUnbounded.cpp | 31 +++++++- 12 files changed, 364 insertions(+), 114 deletions(-) diff --git a/src/Functions/GatherUtils/ArraySourceVisitor.h b/src/Functions/GatherUtils/ArraySourceVisitor.h index 989f9ae918a..0446839325e 100644 --- a/src/Functions/GatherUtils/ArraySourceVisitor.h +++ b/src/Functions/GatherUtils/ArraySourceVisitor.h @@ -19,12 +19,12 @@ struct ConstSource; using NumericArraySources = typename TypeListMap::Type; using BasicArraySources = typename AppendToTypeList::Type; -using NullableArraySources = typename TypeListMap::Type; -using BasicAndNullableArraySources = typename TypeListConcat::Type; -using ConstArraySources = typename TypeListMap::Type; -using TypeListArraySources = typename TypeListConcat::Type; +//using NullableArraySources = typename TypeListMap::Type; +//using BasicAndNullableArraySources = typename TypeListConcat::Type; +//using ConstArraySources = typename TypeListMap::Type; +//using TypeListArraySources = typename TypeListConcat::Type; -class ArraySourceVisitor : public ApplyTypeListForClass::Type +class ArraySourceVisitor : public ApplyTypeListForClass::Type { protected: ~ArraySourceVisitor() = default; diff --git a/src/Functions/GatherUtils/Selectors.h b/src/Functions/GatherUtils/Selectors.h index 9c96b36460d..bbe631a6a3a 100644 --- a/src/Functions/GatherUtils/Selectors.h +++ b/src/Functions/GatherUtils/Selectors.h @@ -32,20 +32,30 @@ void callSelectMemberFunctionWithTupleArgument(Tuple & tuple, Args && ... args) callSelectMemberFunctionWithTupleArgument(tuple, args ..., std::get(tuple)); } +template +void callSelectSource(bool is_const, bool is_nullable, Tuple & tuple, Args && ... args) +{ + if constexpr (index == std::tuple_size::value) + Base::selectSource(is_const, is_nullable, args ...); + else + callSelectSource(is_const, is_nullable, tuple, args ..., std::get(tuple)); +} + template struct ArraySourceSelectorVisitor final : public ArraySourceVisitorImpl> { - explicit ArraySourceSelectorVisitor(Args && ... args) : packed_args(args ...) {} + explicit ArraySourceSelectorVisitor(IArraySource & source, Args && ... args) : packed_args(args ...), array_source(source) {} using Tuple = std::tuple; template void visitImpl(Source & source) { - callSelectMemberFunctionWithTupleArgument(packed_args, source); + callSelectSource(array_source.isConst(), array_source.isNullable(), packed_args, source); } Tuple packed_args; + IArraySource & array_source; }; template @@ -54,7 +64,7 @@ struct ArraySourceSelector template static void select(IArraySource & source, Args && ... args) { - ArraySourceSelectorVisitor visitor(args ...); + ArraySourceSelectorVisitor visitor(source, args ...); source.accept(visitor); } }; @@ -87,56 +97,6 @@ struct ArraySinkSelector } }; - -template -struct ValueSourceSelectorVisitor final : public ValueSourceVisitorImpl> -{ - explicit ValueSourceSelectorVisitor(Args && ... args) : packed_args(args ...) {} - - using Tuple = std::tuple; - - template - void visitImpl(Source & source) - { - callSelectMemberFunctionWithTupleArgument(packed_args, source); - } - - Tuple packed_args; -}; - -template -struct ValueSourceSelector -{ - template - static void select(IValueSource & source, Args && ... args) - { - ValueSourceSelectorVisitor visitor(args ...); - source.accept(visitor); - } -}; - -template -struct ArraySinkSourceSelector -{ - template - static void select(IArraySource & source, IArraySink & sink, Args && ... args) - { - ArraySinkSelector::select(sink, source, args ...); - } - - template - static void selectImpl(Sink && sink, IArraySource & source, Args && ... args) - { - ArraySourceSelector::select(source, sink, args ...); - } - - template - static void selectImpl(Source && source, Sink && sink, Args && ... args) - { - Base::selectSourceSink(source, sink, args ...); - } -}; - template struct ArraySourcePairSelector { @@ -147,15 +107,17 @@ struct ArraySourcePairSelector } template - static void selectImpl(FirstSource && first, IArraySource & second, Args && ... args) + static void selectSource(bool is_const, bool is_nullable, FirstSource && first, IArraySource & second, Args && ... args) { - ArraySourceSelector::select(second, first, args ...); + ArraySourceSelector::select(second, is_const, is_nullable, first, args ...); } template - static void selectImpl(SecondSource && second, FirstSource && first, Args && ... args) + static void selectSource(bool is_second_const, bool is_second_nullable, SecondSource && second, + bool is_first_const, bool is_first_nullable, FirstSource && first, Args && ... args) { - Base::selectSourcePair(first, second, args ...); + Base::selectSourcePair(is_first_const, is_first_nullable, first, + is_second_const, is_second_nullable, second, args ...); } }; diff --git a/src/Functions/GatherUtils/concat.cpp b/src/Functions/GatherUtils/concat.cpp index 3435baf147e..4fe6948ab56 100644 --- a/src/Functions/GatherUtils/concat.cpp +++ b/src/Functions/GatherUtils/concat.cpp @@ -23,37 +23,54 @@ struct ArrayConcat : public ArraySourceSelector { using Sources = std::vector>; +// template +// static void selectSource(Source && source, const Sources & sources, ColumnArray::MutablePtr & result) +// { +// using SourceType = typename std::decay::type; +// using Sink = typename SourceType::SinkType; +// result = ColumnArray::create(source.createValuesColumn()); +// Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); +// +// concat(sources, std::move(sink)); +// } +// +// template +// static void selectImpl(ConstSource && source, const Sources & sources, ColumnArray::MutablePtr & result) +// { +// using SourceType = typename std::decay::type; +// using Sink = typename SourceType::SinkType; +// result = ColumnArray::create(source.createValuesColumn()); +// Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); +// +// concat(sources, std::move(sink)); +// } + template - static void selectImpl(Source && source, const Sources & sources, ColumnArray::MutablePtr & result) + static void selectSource(bool /*is_const*/, bool is_nullable, Source & source, const Sources & sources, ColumnArray::MutablePtr & result) { using SourceType = typename std::decay::type; using Sink = typename SourceType::SinkType; - result = ColumnArray::create(source.createValuesColumn()); - Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); - concat(sources, std::move(sink)); - } + if (is_nullable) + { + using NullableSource = NullableArraySource; + using NullableSink = typename NullableSource::SinkType; - template - static void selectImpl(ConstSource && source, const Sources & sources, ColumnArray::MutablePtr & result) - { - using SourceType = typename std::decay::type; - using Sink = typename SourceType::SinkType; - result = ColumnArray::create(source.createValuesColumn()); - Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + auto & nullable_source = static_cast(source); - concat(sources, std::move(sink)); - } - template - static void selectImpl(ConstSource & source, const Sources & sources, ColumnArray::MutablePtr & result) - { - using SourceType = typename std::decay::type; - using Sink = typename SourceType::SinkType; - result = ColumnArray::create(source.createValuesColumn()); - Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + result = ColumnArray::create(nullable_source.createValuesColumn()); + NullableSink sink(result->getData(), result->getOffsets(), source.getColumnSize()); - concat(sources, std::move(sink)); + concat(sources, std::move(sink)); + } + else + { + result = ColumnArray::create(source.createValuesColumn()); + Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + concat(sources, std::move(sink)); + } } }; diff --git a/src/Functions/GatherUtils/has_all.cpp b/src/Functions/GatherUtils/has_all.cpp index 491631d0c7c..6e34a851c02 100644 --- a/src/Functions/GatherUtils/has_all.cpp +++ b/src/Functions/GatherUtils/has_all.cpp @@ -11,9 +11,53 @@ namespace struct ArrayHasAllSelectArraySourcePair : public ArraySourcePairSelector { template - static void selectSourcePair(FirstSource && first, SecondSource && second, ColumnUInt8 & result) + static void callFunction(FirstSource && first, + bool is_second_const, bool is_second_nullable, SecondSource && second, + ColumnUInt8 & result) { - arrayAllAny(first, second, result); + using SourceType = typename std::decay::type; + + if (is_second_nullable) + { + using NullableSource = NullableArraySource; + + if (is_second_const) + arrayAllAny(first, static_cast &>(second), result); + else + arrayAllAny(first, static_cast(second), result); + } + else + { + if (is_second_const) + arrayAllAny(first, static_cast &>(second), result); + else + arrayAllAny(first, second, result); + } + } + + template + static void selectSourcePair(bool is_first_const, bool is_first_nullable, FirstSource && first, + bool is_second_const, bool is_second_nullable, SecondSource && second, + ColumnUInt8 & result) + { + using SourceType = typename std::decay::type; + + if (is_first_nullable) + { + using NullableSource = NullableArraySource; + + if (is_first_const) + callFunction(static_cast &>(first), is_second_const, is_second_nullable, second, result); + else + callFunction(static_cast(first), is_second_const, is_second_nullable, second, result); + } + else + { + if (is_first_const) + callFunction(static_cast &>(first), is_second_const, is_second_nullable, second, result); + else + callFunction(first, is_second_const, is_second_nullable, second, result); + } } }; diff --git a/src/Functions/GatherUtils/has_any.cpp b/src/Functions/GatherUtils/has_any.cpp index 6ad0eeb8dc0..b7a8c9f620d 100644 --- a/src/Functions/GatherUtils/has_any.cpp +++ b/src/Functions/GatherUtils/has_any.cpp @@ -11,9 +11,53 @@ namespace struct ArrayHasAnySelectArraySourcePair : public ArraySourcePairSelector { template - static void selectSourcePair(FirstSource && first, SecondSource && second, ColumnUInt8 & result) + static void callFunction(FirstSource && first, + bool is_second_const, bool is_second_nullable, SecondSource && second, + ColumnUInt8 & result) { - arrayAllAny(first, second, result); + using SourceType = typename std::decay::type; + + if (is_second_nullable) + { + using NullableSource = NullableArraySource; + + if (is_second_const) + arrayAllAny(first, static_cast &>(second), result); + else + arrayAllAny(first, static_cast(second), result); + } + else + { + if (is_second_const) + arrayAllAny(first, static_cast &>(second), result); + else + arrayAllAny(first, second, result); + } + } + + template + static void selectSourcePair(bool is_first_const, bool is_first_nullable, FirstSource && first, + bool is_second_const, bool is_second_nullable, SecondSource && second, + ColumnUInt8 & result) + { + using SourceType = typename std::decay::type; + + if (is_first_nullable) + { + using NullableSource = NullableArraySource; + + if (is_first_const) + callFunction(static_cast &>(first), is_second_const, is_second_nullable, second, result); + else + callFunction(static_cast(first), is_second_const, is_second_nullable, second, result); + } + else + { + if (is_first_const) + callFunction(static_cast &>(first), is_second_const, is_second_nullable, second, result); + else + callFunction(first, is_second_const, is_second_nullable, second, result); + } } }; diff --git a/src/Functions/GatherUtils/has_substr.cpp b/src/Functions/GatherUtils/has_substr.cpp index fe16c423428..244a1d21633 100644 --- a/src/Functions/GatherUtils/has_substr.cpp +++ b/src/Functions/GatherUtils/has_substr.cpp @@ -11,9 +11,53 @@ namespace struct ArrayHasSubstrSelectArraySourcePair : public ArraySourcePairSelector { template - static void selectSourcePair(FirstSource && first, SecondSource && second, ColumnUInt8 & result) + static void callFunction(FirstSource && first, + bool is_second_const, bool is_second_nullable, SecondSource && second, + ColumnUInt8 & result) { - arrayAllAny(first, second, result); + using SourceType = typename std::decay::type; + + if (is_second_nullable) + { + using NullableSource = NullableArraySource; + + if (is_second_const) + arrayAllAny(first, static_cast &>(second), result); + else + arrayAllAny(first, static_cast(second), result); + } + else + { + if (is_second_const) + arrayAllAny(first, static_cast &>(second), result); + else + arrayAllAny(first, second, result); + } + } + + template + static void selectSourcePair(bool is_first_const, bool is_first_nullable, FirstSource && first, + bool is_second_const, bool is_second_nullable, SecondSource && second, + ColumnUInt8 & result) + { + using SourceType = typename std::decay::type; + + if (is_first_nullable) + { + using NullableSource = NullableArraySource; + + if (is_first_const) + callFunction(static_cast &>(first), is_second_const, is_second_nullable, second, result); + else + callFunction(static_cast(first), is_second_const, is_second_nullable, second, result); + } + else + { + if (is_first_const) + callFunction(static_cast &>(first), is_second_const, is_second_nullable, second, result); + else + callFunction(first, is_second_const, is_second_nullable, second, result); + } } }; diff --git a/src/Functions/GatherUtils/sliceDynamicOffsetBounded.cpp b/src/Functions/GatherUtils/sliceDynamicOffsetBounded.cpp index 5222bf525cc..f8f46a2ac49 100644 --- a/src/Functions/GatherUtils/sliceDynamicOffsetBounded.cpp +++ b/src/Functions/GatherUtils/sliceDynamicOffsetBounded.cpp @@ -13,13 +13,37 @@ namespace struct SliceDynamicOffsetBoundedSelectArraySource : public ArraySourceSelector { template - static void selectImpl(Source && source, const IColumn & offset_column, const IColumn & length_column, ColumnArray::MutablePtr & result) + static void selectSource(bool is_const, bool is_nullable, Source && source, + const IColumn & offset_column, const IColumn & length_column, ColumnArray::MutablePtr & result) { using SourceType = typename std::decay::type; using Sink = typename SourceType::SinkType; - result = ColumnArray::create(source.createValuesColumn()); - Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); - sliceDynamicOffsetBounded(source, sink, offset_column, length_column); + + if (is_nullable) + { + using NullableSource = NullableArraySource; + using NullableSink = typename NullableSource::SinkType; + + auto & nullable_source = static_cast(source); + + result = ColumnArray::create(nullable_source.createValuesColumn()); + NullableSink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + if (is_const) + sliceDynamicOffsetBounded(static_cast &>(source), sink, offset_column, length_column); + else + sliceDynamicOffsetBounded(static_cast(source), sink, offset_column, length_column); + } + else + { + result = ColumnArray::create(source.createValuesColumn()); + Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + if (is_const) + sliceDynamicOffsetBounded(static_cast &>(source), sink, offset_column, length_column); + else + sliceDynamicOffsetBounded(source, sink, offset_column, length_column); + } } }; diff --git a/src/Functions/GatherUtils/sliceDynamicOffsetUnbounded.cpp b/src/Functions/GatherUtils/sliceDynamicOffsetUnbounded.cpp index fa98028f36f..2aa6a8903a4 100644 --- a/src/Functions/GatherUtils/sliceDynamicOffsetUnbounded.cpp +++ b/src/Functions/GatherUtils/sliceDynamicOffsetUnbounded.cpp @@ -14,13 +14,36 @@ struct SliceDynamicOffsetUnboundedSelectArraySource : public ArraySourceSelector { template - static void selectImpl(Source && source, const IColumn & offset_column, ColumnArray::MutablePtr & result) + static void selectSource(bool is_const, bool is_nullable, Source && source, const IColumn & offset_column, ColumnArray::MutablePtr & result) { using SourceType = typename std::decay::type; using Sink = typename SourceType::SinkType; - result = ColumnArray::create(source.createValuesColumn()); - Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); - sliceDynamicOffsetUnbounded(source, sink, offset_column); + + if (is_nullable) + { + using NullableSource = NullableArraySource; + using NullableSink = typename NullableSource::SinkType; + + auto & nullable_source = static_cast(source); + + result = ColumnArray::create(nullable_source.createValuesColumn()); + NullableSink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + if (is_const) + sliceDynamicOffsetUnbounded(static_cast &>(source), sink, offset_column); + else + sliceDynamicOffsetUnbounded(static_cast(source), sink, offset_column); + } + else + { + result = ColumnArray::create(source.createValuesColumn()); + Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + if (is_const) + sliceDynamicOffsetUnbounded(static_cast &>(source), sink, offset_column); + else + sliceDynamicOffsetUnbounded(source, sink, offset_column); + } } }; diff --git a/src/Functions/GatherUtils/sliceFromLeftConstantOffsetBounded.cpp b/src/Functions/GatherUtils/sliceFromLeftConstantOffsetBounded.cpp index 96ea1e076e1..404cfa1b10d 100644 --- a/src/Functions/GatherUtils/sliceFromLeftConstantOffsetBounded.cpp +++ b/src/Functions/GatherUtils/sliceFromLeftConstantOffsetBounded.cpp @@ -14,13 +14,36 @@ struct SliceFromLeftConstantOffsetBoundedSelectArraySource : public ArraySourceSelector { template - static void selectImpl(Source && source, size_t & offset, ssize_t & length, ColumnArray::MutablePtr & result) + static void selectSource(bool is_const, bool is_nullable, Source && source, size_t & offset, ssize_t & length, ColumnArray::MutablePtr & result) { using SourceType = typename std::decay::type; using Sink = typename SourceType::SinkType; - result = ColumnArray::create(source.createValuesColumn()); - Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); - sliceFromLeftConstantOffsetBounded(source, sink, offset, length); + + if (is_nullable) + { + using NullableSource = NullableArraySource; + using NullableSink = typename NullableSource::SinkType; + + auto & nullable_source = static_cast(source); + + result = ColumnArray::create(nullable_source.createValuesColumn()); + NullableSink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + if (is_const) + sliceFromLeftConstantOffsetBounded(static_cast &>(source), sink, offset, length); + else + sliceFromLeftConstantOffsetBounded(static_cast(source), sink, offset, length); + } + else + { + result = ColumnArray::create(source.createValuesColumn()); + Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + if (is_const) + sliceFromLeftConstantOffsetBounded(static_cast &>(source), sink, offset, length); + else + sliceFromLeftConstantOffsetBounded(source, sink, offset, length); + } } }; diff --git a/src/Functions/GatherUtils/sliceFromLeftConstantOffsetUnbounded.cpp b/src/Functions/GatherUtils/sliceFromLeftConstantOffsetUnbounded.cpp index a6b5f799c80..1a7fb03a275 100644 --- a/src/Functions/GatherUtils/sliceFromLeftConstantOffsetUnbounded.cpp +++ b/src/Functions/GatherUtils/sliceFromLeftConstantOffsetUnbounded.cpp @@ -14,13 +14,36 @@ struct SliceFromLeftConstantOffsetUnboundedSelectArraySource : public ArraySourceSelector { template - static void selectImpl(Source && source, size_t & offset, ColumnArray::MutablePtr & result) + static void selectSource(bool is_const, bool is_nullable, Source && source, size_t & offset, ColumnArray::MutablePtr & result) { using SourceType = typename std::decay::type; using Sink = typename SourceType::SinkType; - result = ColumnArray::create(source.createValuesColumn()); - Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); - sliceFromLeftConstantOffsetUnbounded(source, sink, offset); + + if (is_nullable) + { + using NullableSource = NullableArraySource; + using NullableSink = typename NullableSource::SinkType; + + auto & nullable_source = static_cast(source); + + result = ColumnArray::create(nullable_source.createValuesColumn()); + NullableSink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + if (is_const) + sliceFromLeftConstantOffsetUnbounded(static_cast &>(source), sink, offset); + else + sliceFromLeftConstantOffsetUnbounded(static_cast(source), sink, offset); + } + else + { + result = ColumnArray::create(source.createValuesColumn()); + Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + if (is_const) + sliceFromLeftConstantOffsetUnbounded(static_cast &>(source), sink, offset); + else + sliceFromLeftConstantOffsetUnbounded(source, sink, offset); + } } }; diff --git a/src/Functions/GatherUtils/sliceFromRightConstantOffsetBounded.cpp b/src/Functions/GatherUtils/sliceFromRightConstantOffsetBounded.cpp index 35833950cfe..faa6c6fd4e9 100644 --- a/src/Functions/GatherUtils/sliceFromRightConstantOffsetBounded.cpp +++ b/src/Functions/GatherUtils/sliceFromRightConstantOffsetBounded.cpp @@ -14,13 +14,36 @@ struct SliceFromRightConstantOffsetBoundedSelectArraySource : public ArraySourceSelector { template - static void selectImpl(Source && source, size_t & offset, ssize_t & length, ColumnArray::MutablePtr & result) + static void selectSource(bool is_const, bool is_nullable, Source && source, size_t & offset, ssize_t & length, ColumnArray::MutablePtr & result) { using SourceType = typename std::decay::type; using Sink = typename SourceType::SinkType; - result = ColumnArray::create(source.createValuesColumn()); - Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); - sliceFromRightConstantOffsetBounded(source, sink, offset, length); + + if (is_nullable) + { + using NullableSource = NullableArraySource; + using NullableSink = typename NullableSource::SinkType; + + auto & nullable_source = static_cast(source); + + result = ColumnArray::create(nullable_source.createValuesColumn()); + NullableSink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + if (is_const) + sliceFromRightConstantOffsetBounded(static_cast &>(source), sink, offset, length); + else + sliceFromRightConstantOffsetBounded(static_cast(source), sink, offset, length); + } + else + { + result = ColumnArray::create(source.createValuesColumn()); + Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + if (is_const) + sliceFromRightConstantOffsetBounded(static_cast &>(source), sink, offset, length); + else + sliceFromRightConstantOffsetBounded(source, sink, offset, length); + } } }; diff --git a/src/Functions/GatherUtils/sliceFromRightConstantOffsetUnbounded.cpp b/src/Functions/GatherUtils/sliceFromRightConstantOffsetUnbounded.cpp index 945450b4208..59bb0c21599 100644 --- a/src/Functions/GatherUtils/sliceFromRightConstantOffsetUnbounded.cpp +++ b/src/Functions/GatherUtils/sliceFromRightConstantOffsetUnbounded.cpp @@ -14,13 +14,36 @@ struct SliceFromRightConstantOffsetUnboundedSelectArraySource : public ArraySourceSelector { template - static void selectImpl(Source && source, size_t & offset, ColumnArray::MutablePtr & result) + static void selectSource(bool is_const, bool is_nullable, Source && source, size_t & offset, ColumnArray::MutablePtr & result) { using SourceType = typename std::decay::type; using Sink = typename SourceType::SinkType; - result = ColumnArray::create(source.createValuesColumn()); - Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); - sliceFromRightConstantOffsetUnbounded(source, sink, offset); + + if (is_nullable) + { + using NullableSource = NullableArraySource; + using NullableSink = typename NullableSource::SinkType; + + auto & nullable_source = static_cast(source); + + result = ColumnArray::create(nullable_source.createValuesColumn()); + NullableSink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + if (is_const) + sliceFromRightConstantOffsetUnbounded(static_cast &>(source), sink, offset); + else + sliceFromRightConstantOffsetUnbounded(static_cast(source), sink, offset); + } + else + { + result = ColumnArray::create(source.createValuesColumn()); + Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + if (is_const) + sliceFromRightConstantOffsetUnbounded(static_cast &>(source), sink, offset); + else + sliceFromRightConstantOffsetUnbounded(source, sink, offset); + } } }; From 7f629c8185a2b8824970158f881bd7f31b8ffbf4 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 21 Sep 2020 14:12:45 +0300 Subject: [PATCH 250/273] Update DatabaseMemory.cpp --- src/Databases/DatabaseMemory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 221e54ce741..5eacb846d52 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -75,7 +75,7 @@ ASTPtr DatabaseMemory::getCreateTableQueryImpl(const String & table_name, const else return {}; } - return it->second; + return it->second->clone(); } UUID DatabaseMemory::tryGetTableUUID(const String & table_name) const From 2f8d7ae208f1ffc8db62256583dc48078823c6b6 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 21 Sep 2020 15:01:27 +0300 Subject: [PATCH 251/273] fix the conditions --- 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 8585603b010..df9619c28c7 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -155,13 +155,13 @@ function run_tests # are already set, keep those values. if [ "$PR_TO_TEST" -ne 0 ] \ && [ "$(wc -l < changed-test-definitions.txt)" -eq 0 ] \ - && [ "$(wc -l < changed-test-files.txt)" -eq 0 ] + && [ "$(wc -l < changed-test-scripts.txt)" -eq 0 ] then - CHPC_RUNS=${CHPC_RUNS:-13} - CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-0} - else CHPC_RUNS=${CHPC_RUNS:-7} CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-20} + else + CHPC_RUNS=${CHPC_RUNS:-13} + CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-0} fi export CHPC_RUNS export CHPC_MAX_QUERIES From 4dae587e2d3a78208f42c508bebe9b5a36126ada Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 21 Sep 2020 15:02:34 +0300 Subject: [PATCH 252/273] fix test --- tests/performance/columns_hashing.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/columns_hashing.xml b/tests/performance/columns_hashing.xml index 271fff6e543..c45e5d1fe39 100644 --- a/tests/performance/columns_hashing.xml +++ b/tests/performance/columns_hashing.xml @@ -7,6 +7,6 @@ select sum(UserID + 1 in (select UserID from hits_100m_single)) from hits_100m_single select sum((UserID + 1, RegionID) in (select UserID, RegionID from hits_10m_single)) from hits_10m_single select sum(URL in (select URL from hits_10m_single where URL != '')) from hits_10m_single - select sum(MobilePhoneModel in (select MobilePhoneModel from hits_10m_single where MobilePhoneModel != '')) from hits_10m_single + select sum(MobilePhoneModel in (select MobilePhoneModel from hits_100m_single where MobilePhoneModel != '')) from hits_100m_single select sum((MobilePhoneModel, UserID + 1) in (select MobilePhoneModel, UserID from hits_100m_single where MobilePhoneModel != '')) from hits_100m_single From 679c5278e49cd0050ad75d4e7f772cdf89263a95 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 21 Sep 2020 15:14:47 +0300 Subject: [PATCH 253/273] reword the diagnostic about short queries in perf test --- docker/test/performance-comparison/README.md | 2 +- docker/test/performance-comparison/compare.sh | 9 +++++---- docker/test/performance-comparison/report.py | 14 +++++++------- 3 files changed, 13 insertions(+), 12 deletions(-) diff --git a/docker/test/performance-comparison/README.md b/docker/test/performance-comparison/README.md index ec5691488f0..3953e99fc0f 100644 --- a/docker/test/performance-comparison/README.md +++ b/docker/test/performance-comparison/README.md @@ -45,7 +45,7 @@ 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 +#### Unexpected Query Duration 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. diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index f90020f80eb..6a9898ba797 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -688,12 +688,13 @@ create view shortness -- 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. +-- The threshold for 2) is significantly larger than the threshold for 1), to +-- avoid jitter. create table inconsistent_short_marking_report - engine File(TSV, 'report/inconsistent-short-marking.tsv') + engine File(TSV, 'report/unexpected-query-duration.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', + multiIf(marked_short and time > 0.1, '"short" queries must run faster than 0.02 s', + not marked_short and time < 0.02, '"normal" queries must run longer than 0.1 s', '') problem, marked_short, time, test, query_index, query_display_name diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index e8481f77340..8304aa55fc2 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -98,9 +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), +.unexpected-query-duration tr :nth-child(2), +.unexpected-query-duration tr :nth-child(3), +.unexpected-query-duration 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), @@ -325,13 +325,13 @@ if args.report == 'main': 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/inconsistent-short-marking.tsv') + unmarked_short_rows = tsvRows('report/unexpected-query-duration.tsv') error_tests += len(unmarked_short_rows) - addSimpleTable('Inconsistent Short Marking', - ['Problem', 'Is marked as short', 'New client time, s', 'Test', '#', 'Query'], + addSimpleTable('Unexpected Query Duration', + ['Problem', 'Marked as "short"?', 'Run time, s', 'Test', '#', 'Query'], unmarked_short_rows) if unmarked_short_rows: - errors_explained.append([f'Some queries have inconsistent short marking']); + errors_explained.append([f'Some queries have unexpected duration']); def add_partial(): rows = tsvRows('report/partial-queries-report.tsv') From fd2cc343e6fd84c1a1e3da79173acc0ea361819b Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 21 Sep 2020 15:37:34 +0300 Subject: [PATCH 254/273] Update renameat2.cpp --- src/Common/renameat2.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/renameat2.cpp b/src/Common/renameat2.cpp index 02acd8331d2..24e414122dc 100644 --- a/src/Common/renameat2.cpp +++ b/src/Common/renameat2.cpp @@ -81,7 +81,7 @@ static bool renameat2(const std::string & old_path, const std::string & new_path static bool renameat2(const std::string &, const std::string &, int) { - return false + return false; } #endif From 741912cafd2814088202b4ca740bcff25bf2cf7c Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 21 Sep 2020 21:03:17 +0800 Subject: [PATCH 255/273] Don't hide interface symbols --- src/Functions/GatherUtils/IArraySink.h | 3 ++- src/Functions/GatherUtils/IArraySource.h | 3 ++- src/Functions/GatherUtils/IValueSource.h | 3 ++- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Functions/GatherUtils/IArraySink.h b/src/Functions/GatherUtils/IArraySink.h index a41d99b5f6a..8bc3f05edd1 100644 --- a/src/Functions/GatherUtils/IArraySink.h +++ b/src/Functions/GatherUtils/IArraySink.h @@ -13,7 +13,6 @@ namespace ErrorCodes namespace GatherUtils { -#pragma GCC visibility push(hidden) struct IArraySink { @@ -25,6 +24,8 @@ struct IArraySink } }; +#pragma GCC visibility push(hidden) + template class ArraySinkImpl : public Visitable {}; diff --git a/src/Functions/GatherUtils/IArraySource.h b/src/Functions/GatherUtils/IArraySource.h index 386108f038e..c34c3ddb683 100644 --- a/src/Functions/GatherUtils/IArraySource.h +++ b/src/Functions/GatherUtils/IArraySource.h @@ -13,7 +13,6 @@ namespace ErrorCodes namespace GatherUtils { -#pragma GCC visibility push(hidden) struct IArraySource { @@ -31,6 +30,8 @@ struct IArraySource } }; +#pragma GCC visibility push(hidden) + template class ArraySourceImpl : public Visitable {}; diff --git a/src/Functions/GatherUtils/IValueSource.h b/src/Functions/GatherUtils/IValueSource.h index 55af364a3a9..2be44143c84 100644 --- a/src/Functions/GatherUtils/IValueSource.h +++ b/src/Functions/GatherUtils/IValueSource.h @@ -13,7 +13,6 @@ namespace ErrorCodes namespace GatherUtils { -#pragma GCC visibility push(hidden) struct IValueSource { @@ -27,6 +26,8 @@ struct IValueSource virtual bool isConst() const { return false; } }; +#pragma GCC visibility push(hidden) + template class ValueSourceImpl : public Visitable {}; From c77a6967a3a9615b372d4f1d0d5c07ec5fa850a2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 21 Sep 2020 19:22:23 +0300 Subject: [PATCH 256/273] Remove commented code. --- .../GatherUtils/ArraySourceVisitor.h | 4 ---- src/Functions/GatherUtils/concat.cpp | 22 ------------------- 2 files changed, 26 deletions(-) diff --git a/src/Functions/GatherUtils/ArraySourceVisitor.h b/src/Functions/GatherUtils/ArraySourceVisitor.h index 0446839325e..260e9a87161 100644 --- a/src/Functions/GatherUtils/ArraySourceVisitor.h +++ b/src/Functions/GatherUtils/ArraySourceVisitor.h @@ -19,10 +19,6 @@ struct ConstSource; using NumericArraySources = typename TypeListMap::Type; using BasicArraySources = typename AppendToTypeList::Type; -//using NullableArraySources = typename TypeListMap::Type; -//using BasicAndNullableArraySources = typename TypeListConcat::Type; -//using ConstArraySources = typename TypeListMap::Type; -//using TypeListArraySources = typename TypeListConcat::Type; class ArraySourceVisitor : public ApplyTypeListForClass::Type { diff --git a/src/Functions/GatherUtils/concat.cpp b/src/Functions/GatherUtils/concat.cpp index 4fe6948ab56..d73c98a0e88 100644 --- a/src/Functions/GatherUtils/concat.cpp +++ b/src/Functions/GatherUtils/concat.cpp @@ -23,28 +23,6 @@ struct ArrayConcat : public ArraySourceSelector { using Sources = std::vector>; -// template -// static void selectSource(Source && source, const Sources & sources, ColumnArray::MutablePtr & result) -// { -// using SourceType = typename std::decay::type; -// using Sink = typename SourceType::SinkType; -// result = ColumnArray::create(source.createValuesColumn()); -// Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); -// -// concat(sources, std::move(sink)); -// } -// -// template -// static void selectImpl(ConstSource && source, const Sources & sources, ColumnArray::MutablePtr & result) -// { -// using SourceType = typename std::decay::type; -// using Sink = typename SourceType::SinkType; -// result = ColumnArray::create(source.createValuesColumn()); -// Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); -// -// concat(sources, std::move(sink)); -// } - template static void selectSource(bool /*is_const*/, bool is_nullable, Source & source, const Sources & sources, ColumnArray::MutablePtr & result) { From 10d3a1d730f6f08fe9f1d8403a209980f7849408 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 21 Sep 2020 20:49:15 +0300 Subject: [PATCH 257/273] Update ThreadPool.cpp --- src/Common/ThreadPool.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index f1fa82b7e68..56198b97be5 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -233,7 +233,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ std::is_same_v ? CurrentMetrics::GlobalThreadActive : CurrentMetrics::LocalThreadActive); job(); - job = Job(); + job = {}; } catch (...) { From 6e596b9c7d5673372d3bf1eeed5b6e4df6194e0c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 21 Sep 2020 21:15:41 +0300 Subject: [PATCH 258/273] Added a test for Enum conversion in Distributed table if one shard is local --- ...buted_local_type_conversion_enum.reference | 21 +++++++++++++++++ ...distributed_local_type_conversion_enum.sql | 23 +++++++++++++++++++ .../queries/0_stateless/arcadia_skip_list.txt | 1 + 3 files changed, 45 insertions(+) create mode 100644 tests/queries/0_stateless/01505_distributed_local_type_conversion_enum.reference create mode 100644 tests/queries/0_stateless/01505_distributed_local_type_conversion_enum.sql diff --git a/tests/queries/0_stateless/01505_distributed_local_type_conversion_enum.reference b/tests/queries/0_stateless/01505_distributed_local_type_conversion_enum.reference new file mode 100644 index 00000000000..94a9822a8df --- /dev/null +++ b/tests/queries/0_stateless/01505_distributed_local_type_conversion_enum.reference @@ -0,0 +1,21 @@ +abc +def +ghi +--- +abc +def +xyz +--- +abc +def +xyz +abc +def +xyz +--- +abc +def +xyz +abc +def +xyz diff --git a/tests/queries/0_stateless/01505_distributed_local_type_conversion_enum.sql b/tests/queries/0_stateless/01505_distributed_local_type_conversion_enum.sql new file mode 100644 index 00000000000..6f62716d312 --- /dev/null +++ b/tests/queries/0_stateless/01505_distributed_local_type_conversion_enum.sql @@ -0,0 +1,23 @@ +DROP TABLE IF EXISTS t; +DROP TABLE IF EXISTS d; + +CREATE TABLE t (x Enum8('abc' = 0, 'def' = 1, 'ghi' = 2)) ENGINE = TinyLog; +INSERT INTO t VALUES (0), (1), (2); +SELECT * FROM t; + +SELECT '---'; +CREATE TABLE d (x Enum8('abc' = 0, 'def' = 1, 'xyz' = 2)) ENGINE = Distributed(test_shard_localhost, currentDatabase(), t); +SELECT * FROM d; +DROP TABLE d; + +SELECT '---'; +CREATE TABLE d (x Enum8('abc' = 0, 'def' = 1, 'xyz' = 2)) ENGINE = Distributed(test_cluster_two_shards, currentDatabase(), t); +SELECT * FROM d; +DROP TABLE d; + +SELECT '---'; +CREATE TABLE d (x Enum8('abc' = 0, 'def' = 1, 'xyz' = 2)) ENGINE = Distributed(test_cluster_two_shards_localhost, currentDatabase(), t); +SELECT * FROM d; +DROP TABLE d; + +DROP TABLE t; diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index d776f8d80d6..69391ca9fd4 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -144,3 +144,4 @@ 01473_event_time_microseconds 01461_query_start_time_microseconds 01455_shard_leaf_max_rows_bytes_to_read +01505_distributed_local_type_conversion_enum From 4696e2279971b0050ff256187305be15050412fb Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Mon, 21 Sep 2020 22:08:41 +0300 Subject: [PATCH 259/273] 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 a37ef553cfb..79cdc8ea8d2 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -277,7 +277,7 @@ for query_index in queries_to_run: # 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.elapsed} s)', file=sys.stderr) + print(f'The query no. {query_index} is taking too long to run ({elapsed} s)', file=sys.stderr) exit(2) # Be careful with the counter, after this line it's the next iteration From 40ff1cf9127eca5c6ab8b6bfabd689f28fea5fd3 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 22 Sep 2020 00:45:53 +0300 Subject: [PATCH 260/273] Update adopters.md --- docs/en/introduction/adopters.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 596fe20be90..6d57dfde9cd 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -38,6 +38,7 @@ toc_title: Adopters | Deutsche Bank | Finance | BI Analytics | — | — | [Slides in English, October 2019](https://bigdatadays.ru/wp-content/uploads/2019/10/D2-H3-3_Yakunin-Goihburg.pdf) | | Diva-e | Digital consulting | Main Product | — | — | [Slides in English, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup29/ClickHouse-MeetUp-Unusual-Applications-sd-2019-09-17.pdf) | | Ecwid | E-commerce SaaS | Metrics, Logging | — | — | [Slides in Russian, April 2019](https://nastachku.ru/var/files/1/presentation/backend/2_Backend_6.pdf) | +| eBay | E-commerce | TBA | — | — | [Webinar, Sep 2020](https://altinity.com/webinarspage/2020/09/08/migrating-from-druid-to-next-gen-olap-on-clickhouse-ebays-experience) | | Exness | Trading | Metrics, Logging | — | — | [Talk in Russian, May 2019](https://youtu.be/_rpU-TvSfZ8?t=3215) | | FastNetMon | DDoS Protection | Main Product | | — | [Official website](https://fastnetmon.com/docs-fnm-advanced/fastnetmon-advanced-traffic-persistency/) | | Flipkart | e-Commerce | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=239) | From 3c1a679612185e21a73847fd72ba6361496858e6 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 22 Sep 2020 02:06:19 +0300 Subject: [PATCH 261/273] Update performance-test.md --- docs/en/operations/performance-test.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/performance-test.md b/docs/en/operations/performance-test.md index c3ef10da774..734e63b0b35 100644 --- a/docs/en/operations/performance-test.md +++ b/docs/en/operations/performance-test.md @@ -13,9 +13,9 @@ With this instruction you can run basic ClickHouse performance test on any serve 4. ssh to the server and download it with wget: ```bash # For amd64: -wget https://clickhouse-builds.s3.yandex.net/0/00ba767f5d2a929394ea3be193b1f79074a1c4bc/1578163263_binary/clickhouse +wget https://clickhouse-builds.s3.yandex.net/0/e29c4c3cc47ab2a6c4516486c1b77d57e7d42643/clickhouse_build_check/gcc-10_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse # For aarch64: -wget https://clickhouse-builds.s3.yandex.net/0/00ba767f5d2a929394ea3be193b1f79074a1c4bc/1578161264_binary/clickhouse +wget https://clickhouse-builds.s3.yandex.net/0/e29c4c3cc47ab2a6c4516486c1b77d57e7d42643/clickhouse_special_build_check/clang-10-aarch64_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse # Then do: chmod a+x clickhouse ``` From df122b90b4dab10d078a41032ed1c3c09513e15b Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 22 Sep 2020 02:07:12 +0300 Subject: [PATCH 262/273] Update performance-test.md --- docs/en/operations/performance-test.md | 22 +++++++--------------- 1 file changed, 7 insertions(+), 15 deletions(-) diff --git a/docs/en/operations/performance-test.md b/docs/en/operations/performance-test.md index 734e63b0b35..984bbe02174 100644 --- a/docs/en/operations/performance-test.md +++ b/docs/en/operations/performance-test.md @@ -19,43 +19,35 @@ wget https://clickhouse-builds.s3.yandex.net/0/e29c4c3cc47ab2a6c4516486c1b77d57e # Then do: chmod a+x clickhouse ``` -5. Download configs: -```bash -wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/config.xml -wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/users.xml -mkdir config.d -wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/config.d/path.xml -O config.d/path.xml -wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/config.d/log_to_console.xml -O config.d/log_to_console.xml -``` -6. Download benchmark files: +5. Download benchmark files: ```bash wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/clickhouse/benchmark-new.sh chmod a+x benchmark-new.sh wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/clickhouse/queries.sql ``` -7. Download test data according to the [Yandex.Metrica dataset](../getting-started/example-datasets/metrica.md) instruction (“hits” table containing 100 million rows). +6. Download test data according to the [Yandex.Metrica dataset](../getting-started/example-datasets/metrica.md) instruction (“hits” table containing 100 million rows). ```bash wget https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_100m_obfuscated_v1.tar.xz tar xvf hits_100m_obfuscated_v1.tar.xz -C . mv hits_100m_obfuscated_v1/* . ``` -8. Run the server: +7. Run the server: ```bash ./clickhouse server ``` -9. Check the data: ssh to the server in another terminal +8. Check the data: ssh to the server in another terminal ```bash ./clickhouse client --query "SELECT count() FROM hits_100m_obfuscated" 100000000 ``` -10. Edit the benchmark-new.sh, change `clickhouse-client` to `./clickhouse client` and add `--max_memory_usage 100000000000` parameter. +9. Edit the benchmark-new.sh, change `clickhouse-client` to `./clickhouse client` and add `--max_memory_usage 100000000000` parameter. ```bash mcedit benchmark-new.sh ``` -11. Run the benchmark: +10. Run the benchmark: ```bash ./benchmark-new.sh hits_100m_obfuscated ``` -12. Send the numbers and the info about your hardware configuration to clickhouse-feedback@yandex-team.com +11. Send the numbers and the info about your hardware configuration to clickhouse-feedback@yandex-team.com All the results are published here: https://clickhouse.tech/benchmark/hardware/ From f9402c95fa4599d94d7240601f989005bb2e15fb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 22 Sep 2020 04:17:40 +0300 Subject: [PATCH 263/273] Add benchmark results from seo.do --- website/benchmark/hardware/index.html | 3 +- .../hardware/results/052_amd_epyc_7642.json | 56 +++++++++++++++++++ 2 files changed, 58 insertions(+), 1 deletion(-) create mode 100644 website/benchmark/hardware/results/052_amd_epyc_7642.json diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index ce7fb602258..68bc4b29653 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -68,7 +68,8 @@ Results for Yandex Managed ClickHouse for "cold cache" are biased and should not Results for AWS Lightsail is from Vamsi Krishna B.
Results for Dell XPS laptop and Google Pixel phone is from Alexander Kuzmenkov.
Results for Android phones for "cold cache" are done without cache flushing, so they are not "cold" and cannot be compared.
-Results for Digital Ocean are from Zimin Aleksey. +Results for Digital Ocean are from Zimin Aleksey.
+Results for 2x EPYC 7642 w/ 512 GB RAM (192 Cores) + 12X 1TB SSD (RAID6) are from Yiğit Konur and Metehan Çetinkaya of seo.do.

diff --git a/website/benchmark/hardware/results/052_amd_epyc_7642.json b/website/benchmark/hardware/results/052_amd_epyc_7642.json new file mode 100644 index 00000000000..b60146d515f --- /dev/null +++ b/website/benchmark/hardware/results/052_amd_epyc_7642.json @@ -0,0 +1,56 @@ +[ + { + "system": "AMD EPYC 7642", + "system_full": "2x AMD EPYC 7642 / 512 GB RAM / 12x 1TB SSD (RAID 6)", + "time": "2020-09-21 00:00:00", + "kind": "server", + "result": + [ + [0.003, 0.003, 0.002], + [0.039, 0.041, 0.024], + [0.052, 0.029, 0.029], + [0.087, 0.031, 0.032], + [0.152, 0.106, 0.105], + [0.204, 0.128, 0.128], + [0.049, 0.028, 0.027], + [0.031, 0.024, 0.027], + [0.190, 0.130, 0.125], + [0.210, 0.142, 0.138], + [0.142, 0.091, 0.087], + [0.143, 0.101, 0.097], + [0.318, 0.170, 0.163], + [0.303, 0.193, 0.191], + [0.240, 0.175, 0.166], + [0.200, 0.166, 0.161], + [0.466, 0.364, 0.345], + [0.298, 0.244, 0.231], + [1.288, 0.901, 0.859], + [0.087, 0.031, 0.025], + [0.663, 0.201, 0.191], + [0.661, 0.213, 0.154], + [1.118, 0.599, 0.593], + [1.708, 0.392, 0.318], + [0.202, 0.065, 0.066], + [0.135, 0.061, 0.057], + [0.203, 0.066, 0.067], + [0.630, 0.296, 0.290], + [0.578, 0.281, 0.262], + [0.662, 0.670, 0.639], + [0.241, 0.153, 0.150], + [0.424, 0.235, 0.231], + [1.505, 1.090, 1.090], + [1.038, 0.818, 0.799], + [1.064, 0.856, 0.809], + [0.332, 0.297, 0.275], + [0.200, 0.169, 0.168], + [0.083, 0.070, 0.071], + [0.090, 0.059, 0.063], + [0.416, 0.419, 0.398], + [0.048, 0.032, 0.032], + [0.036, 0.027, 0.025], + [0.007, 0.007, 0.007] + ] + } +] + + From 7db544e241e131f62493c71bc73b2b33912d673c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 22 Sep 2020 05:19:11 +0300 Subject: [PATCH 264/273] Added a script to perform hardware benchmark in a single command --- benchmark/hardware.sh | 102 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 102 insertions(+) create mode 100755 benchmark/hardware.sh diff --git a/benchmark/hardware.sh b/benchmark/hardware.sh new file mode 100755 index 00000000000..3fd230ec73c --- /dev/null +++ b/benchmark/hardware.sh @@ -0,0 +1,102 @@ +#!/bin/bash -e + +mkdir -p clickhouse-benchmark +pushd clickhouse-benchmark + +TABLE="hits_100m_obfuscated" +DATASET="${TABLE}_v1.tar.xz" +QUERIES_FILE="queries.sql" +TRIES=3 + +FASTER_DOWNLOAD=wget +if command -v axel >/dev/null; then + FASTER_DOWNLOAD=axel +fi + +if [[ ! -f clickhouse ]]; then + CPU=$(uname -m) + if [[ ($CPU == x86_64) || ($CPU == amd64) ]]; then + $FASTER_DOWNLOAD "https://clickhouse-builds.s3.yandex.net/0/e29c4c3cc47ab2a6c4516486c1b77d57e7d42643/clickhouse_build_check/gcc-10_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse" + elif [[ $CPU == aarch64 ]]; then + $FASTER_DOWNLOAD "https://clickhouse-builds.s3.yandex.net/0/e29c4c3cc47ab2a6c4516486c1b77d57e7d42643/clickhouse_special_build_check/clang-10-aarch64_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse" + else + echo "Unsupported CPU type: $CPU" + exit 1 + fi +fi + +chmod a+x clickhouse + +if [[ ! -f $QUERIES_FILE ]]; then + wget "https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/clickhouse/$QUERIES_FILE" +fi + +if [[ ! -d data ]]; then + if [[ ! -f $DATASET ]]; then + $FASTER_DOWNLOAD "https://clickhouse-datasets.s3.yandex.net/hits/partitions/$DATASET" + fi + + tar --strip-components=1 --directory=. -x -v -f $DATASET +fi + +echo "Starting clickhouse-server" + +./clickhouse server > server.log 2>&1 & +PID=$! + +function finish { + kill $PID + wait +} +trap finish EXIT + +echo "Waiting for clickhouse-server to start" + +for i in {1..30}; do + sleep 1 + ./clickhouse client --query "SELECT 'The dataset size is: ', count() FROM $TABLE" && break + if [[ $i == 30 ]]; then exit 1; fi +done + +echo +echo "Will perform benchmark. Results:" +echo + +cat "$QUERIES_FILE" | sed "s/{table}/${TABLE}/g" | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null + + echo -n "[" + for i in $(seq 1 $TRIES); do + RES=$(./clickhouse client --max_memory_usage 100000000000 --time --format=Null --query="$query" 2>&1) + [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" + [[ "$i" != $TRIES ]] && echo -n ", " + done + echo "]," +done + + +echo +echo "Benchmark complete. System info:" + +echo '----version and build id--------' +./clickhouse local --query "SELECT version(), buildId()" +echo '----CPU-------------------------' +lscpu +echo '----Block Devices---------------' +lsblk +echo '----Disk Free and Total--------' +df -h . +echo '----Memory Free and Total-------' +free -h +echo '----Physical Memory Amount------' +cat /proc/meminfo | grep MemTotal +echo '----RAID Info-------------------' +cat /proc/mdstat +#echo '----PCI-------------------------' +#lspci +#echo '----All Hardware Info-----------' +#lshw +echo '--------------------------------' + +echo From ddfd5122414ca498ad0792ccf250beee4bcb02fe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 22 Sep 2020 05:22:53 +0300 Subject: [PATCH 265/273] Adjustments --- benchmark/hardware.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/benchmark/hardware.sh b/benchmark/hardware.sh index 3fd230ec73c..71cd836f2d0 100755 --- a/benchmark/hardware.sh +++ b/benchmark/hardware.sh @@ -54,7 +54,7 @@ echo "Waiting for clickhouse-server to start" for i in {1..30}; do sleep 1 - ./clickhouse client --query "SELECT 'The dataset size is: ', count() FROM $TABLE" && break + ./clickhouse client --query "SELECT 'The dataset size is: ', count() FROM $TABLE" 2>/dev/null && break || echo '.' if [[ $i == 30 ]]; then exit 1; fi done @@ -78,8 +78,9 @@ done echo echo "Benchmark complete. System info:" +echo -echo '----version and build id--------' +echo '----Version and build id--------' ./clickhouse local --query "SELECT version(), buildId()" echo '----CPU-------------------------' lscpu From 923724c0f4946fb24691cae6f7fc3870e2773a81 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 22 Sep 2020 06:06:08 +0300 Subject: [PATCH 266/273] Improvements --- benchmark/hardware.sh | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/benchmark/hardware.sh b/benchmark/hardware.sh index 71cd836f2d0..dc75cc79626 100755 --- a/benchmark/hardware.sh +++ b/benchmark/hardware.sh @@ -11,6 +11,14 @@ TRIES=3 FASTER_DOWNLOAD=wget if command -v axel >/dev/null; then FASTER_DOWNLOAD=axel +else + echo "It's recommended to install 'axel' for faster downloads." +fi + +if command -v pixz >/dev/null; then + TAR_PARAMS='-Ipixz' +else + echo "It's recommended to install 'pixz' for faster decompression of the dataset." fi if [[ ! -f clickhouse ]]; then @@ -36,7 +44,7 @@ if [[ ! -d data ]]; then $FASTER_DOWNLOAD "https://clickhouse-datasets.s3.yandex.net/hits/partitions/$DATASET" fi - tar --strip-components=1 --directory=. -x -v -f $DATASET + tar $TAR_PARAMS --strip-components=1 --directory=. -x -v -f $DATASET fi echo "Starting clickhouse-server" From 2f09e36778a16e4a2527f17c8dbe7fe927ac8f4d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 22 Sep 2020 06:13:28 +0300 Subject: [PATCH 267/273] Allow scale 1000 in benchmark --- benchmark/hardware.sh | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/benchmark/hardware.sh b/benchmark/hardware.sh index dc75cc79626..6d2834d7164 100755 --- a/benchmark/hardware.sh +++ b/benchmark/hardware.sh @@ -1,9 +1,12 @@ #!/bin/bash -e -mkdir -p clickhouse-benchmark -pushd clickhouse-benchmark +if [[ -n $1 ]]; then + SCALE=$1 +else + SCALE=100 +fi -TABLE="hits_100m_obfuscated" +TABLE="hits_${SCALE}m_obfuscated" DATASET="${TABLE}_v1.tar.xz" QUERIES_FILE="queries.sql" TRIES=3 @@ -21,6 +24,9 @@ else echo "It's recommended to install 'pixz' for faster decompression of the dataset." fi +mkdir -p clickhouse-benchmark-$SCALE +pushd clickhouse-benchmark-$SCALE + if [[ ! -f clickhouse ]]; then CPU=$(uname -m) if [[ ($CPU == x86_64) || ($CPU == amd64) ]]; then From 4cd7de175e9402c97cf686a85ab22c6ed69fc44e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 22 Sep 2020 06:17:09 +0300 Subject: [PATCH 268/273] Adjustments --- benchmark/hardware.sh | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/benchmark/hardware.sh b/benchmark/hardware.sh index 6d2834d7164..0ff71df19ae 100755 --- a/benchmark/hardware.sh +++ b/benchmark/hardware.sh @@ -11,6 +11,9 @@ DATASET="${TABLE}_v1.tar.xz" QUERIES_FILE="queries.sql" TRIES=3 +AMD64_BIN_URL="https://clickhouse-builds.s3.yandex.net/0/e29c4c3cc47ab2a6c4516486c1b77d57e7d42643/clickhouse_build_check/gcc-10_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse" +AARCH64_BIN_URL="https://clickhouse-builds.s3.yandex.net/0/e29c4c3cc47ab2a6c4516486c1b77d57e7d42643/clickhouse_special_build_check/clang-10-aarch64_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse" + FASTER_DOWNLOAD=wget if command -v axel >/dev/null; then FASTER_DOWNLOAD=axel @@ -30,9 +33,9 @@ pushd clickhouse-benchmark-$SCALE if [[ ! -f clickhouse ]]; then CPU=$(uname -m) if [[ ($CPU == x86_64) || ($CPU == amd64) ]]; then - $FASTER_DOWNLOAD "https://clickhouse-builds.s3.yandex.net/0/e29c4c3cc47ab2a6c4516486c1b77d57e7d42643/clickhouse_build_check/gcc-10_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse" + $FASTER_DOWNLOAD "$AMD64_BIN_URL" elif [[ $CPU == aarch64 ]]; then - $FASTER_DOWNLOAD "https://clickhouse-builds.s3.yandex.net/0/e29c4c3cc47ab2a6c4516486c1b77d57e7d42643/clickhouse_special_build_check/clang-10-aarch64_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse" + $FASTER_DOWNLOAD "$AARCH64_BIN_URL" else echo "Unsupported CPU type: $CPU" exit 1 From a5b45dabf20f7eed913f69f061f06790eddc9a1c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 22 Sep 2020 10:14:12 +0300 Subject: [PATCH 269/273] Update version_date.tsv after release 20.9.2.20 --- 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 3ec9ee11b95..75605968a37 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v20.9.2.20-stable 2020-09-22 v20.8.3.18-stable 2020-09-18 v20.8.2.3-stable 2020-09-08 v20.7.3.7-stable 2020-09-18 From 2280482530b2dbb6d20073b22fb57a65ccb9e032 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Tue, 22 Sep 2020 13:16:22 +0300 Subject: [PATCH 270/273] Keep LowCardinality in MergeJoin left key result type (#15088) --- src/Interpreters/MergeJoin.cpp | 4 +- src/Interpreters/join_common.cpp | 15 ++++- src/Interpreters/join_common.h | 3 +- ...erge_join_lc_and_nullable_in_key.reference | 29 +++++++++ ...1142_merge_join_lc_and_nullable_in_key.sql | 48 ++++++++++++++ .../01477_lc_in_merge_join_left_key.reference | 35 ++++++++++ .../01477_lc_in_merge_join_left_key.sql | 65 +++++++++++++++++++ 7 files changed, 195 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/01142_merge_join_lc_and_nullable_in_key.reference create mode 100644 tests/queries/0_stateless/01142_merge_join_lc_and_nullable_in_key.sql create mode 100644 tests/queries/0_stateless/01477_lc_in_merge_join_left_key.reference create mode 100644 tests/queries/0_stateless/01477_lc_in_merge_join_left_key.sql diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 0154f8453b3..c9072ec3480 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -602,7 +602,7 @@ void MergeJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) { JoinCommon::checkTypesOfKeys(block, table_join->keyNamesLeft(), right_table_keys, table_join->keyNamesRight()); materializeBlockInplace(block); - JoinCommon::removeLowCardinalityInplace(block, table_join->keyNamesLeft()); + JoinCommon::removeLowCardinalityInplace(block, table_join->keyNamesLeft(), false); sortBlock(block, left_sort_description); @@ -636,6 +636,8 @@ void MergeJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) /// Back thread even with no data. We have some unfinished data in buffer. if (!not_processed && left_blocks_buffer) not_processed = std::make_shared(NotProcessed{{}, 0, 0, 0}); + + JoinCommon::restoreLowCardinalityInplace(block); } template diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index 866893fa359..17c289b151d 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -185,13 +185,24 @@ void removeLowCardinalityInplace(Block & block) } } -void removeLowCardinalityInplace(Block & block, const Names & names) +void removeLowCardinalityInplace(Block & block, const Names & names, bool change_type) { for (const String & column_name : names) { auto & col = block.getByName(column_name); col.column = recursiveRemoveLowCardinality(col.column); - col.type = recursiveRemoveLowCardinality(col.type); + if (change_type) + col.type = recursiveRemoveLowCardinality(col.type); + } +} + +void restoreLowCardinalityInplace(Block & block) +{ + for (size_t i = 0; i < block.columns(); ++i) + { + auto & col = block.getByPosition(i); + if (col.type->lowCardinality() && col.column && !col.column->lowCardinality()) + col.column = changeLowCardinality(col.column, col.type->createColumn()); } } diff --git a/src/Interpreters/join_common.h b/src/Interpreters/join_common.h index 11fecd4e3fb..cfd727704a0 100644 --- a/src/Interpreters/join_common.h +++ b/src/Interpreters/join_common.h @@ -23,7 +23,8 @@ Columns materializeColumns(const Block & block, const Names & names); ColumnRawPtrs materializeColumnsInplace(Block & block, const Names & names); ColumnRawPtrs getRawPointers(const Columns & columns); void removeLowCardinalityInplace(Block & block); -void removeLowCardinalityInplace(Block & block, const Names & names); +void removeLowCardinalityInplace(Block & block, const Names & names, bool change_type = true); +void restoreLowCardinalityInplace(Block & block); ColumnRawPtrs extractKeysForJoin(const Block & block_keys, const Names & key_names_right); diff --git a/tests/queries/0_stateless/01142_merge_join_lc_and_nullable_in_key.reference b/tests/queries/0_stateless/01142_merge_join_lc_and_nullable_in_key.reference new file mode 100644 index 00000000000..d1b29b46df6 --- /dev/null +++ b/tests/queries/0_stateless/01142_merge_join_lc_and_nullable_in_key.reference @@ -0,0 +1,29 @@ +1 l \N Nullable(String) +2 \N Nullable(String) +1 l \N Nullable(String) +2 \N Nullable(String) +- +1 l \N Nullable(String) +0 \N Nullable(String) +0 \N Nullable(String) +1 l \N Nullable(String) +- +1 l \N Nullable(String) +0 \N Nullable(String) +0 \N Nullable(String) +1 l \N Nullable(String) +- +1 l \N Nullable(String) +2 \N Nullable(String) +1 l \N Nullable(String) +2 \N Nullable(String) +- +1 l \N Nullable(String) +\N \N Nullable(String) +1 l \N Nullable(String) +\N \N Nullable(String) +- +1 l \N Nullable(String) +\N \N Nullable(String) +1 l \N Nullable(String) +\N \N Nullable(String) diff --git a/tests/queries/0_stateless/01142_merge_join_lc_and_nullable_in_key.sql b/tests/queries/0_stateless/01142_merge_join_lc_and_nullable_in_key.sql new file mode 100644 index 00000000000..8a1601e3faa --- /dev/null +++ b/tests/queries/0_stateless/01142_merge_join_lc_and_nullable_in_key.sql @@ -0,0 +1,48 @@ +SET join_algorithm = 'partial_merge'; + +DROP TABLE IF EXISTS t; +DROP TABLE IF EXISTS nr; + +CREATE TABLE t (`x` UInt32, `lc` LowCardinality(String)) ENGINE = Memory; +CREATE TABLE nr (`x` Nullable(UInt32), `lc` Nullable(String)) ENGINE = Memory; + +INSERT INTO t VALUES (1, 'l'); +INSERT INTO nr VALUES (2, NULL); + +SET join_use_nulls = 0; + +SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l LEFT JOIN nr AS r USING (x) ORDER BY x; +SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l RIGHT JOIN nr AS r USING (x) ORDER BY x; +SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l FULL JOIN nr AS r USING (x) ORDER BY x; + +SELECT '-'; + +SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x; +SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x; +SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x; + +SELECT '-'; + +SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x; +SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x; +SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x; + +SELECT '-'; + +SET join_use_nulls = 1; + +SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l LEFT JOIN nr AS r USING (x) ORDER BY x; +SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l RIGHT JOIN nr AS r USING (x) ORDER BY x; +SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l FULL JOIN nr AS r USING (x) ORDER BY x; + +SELECT '-'; + +SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x; +SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x; +SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x; + +SELECT '-'; + +SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x; +SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x; +SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x; diff --git a/tests/queries/0_stateless/01477_lc_in_merge_join_left_key.reference b/tests/queries/0_stateless/01477_lc_in_merge_join_left_key.reference new file mode 100644 index 00000000000..0612b4ca23e --- /dev/null +++ b/tests/queries/0_stateless/01477_lc_in_merge_join_left_key.reference @@ -0,0 +1,35 @@ +1 l \N LowCardinality(String) Nullable(String) +2 \N LowCardinality(String) Nullable(String) +1 l \N LowCardinality(String) Nullable(String) +2 \N LowCardinality(String) Nullable(String) +- +0 \N Nullable(String) LowCardinality(String) +1 \N l Nullable(String) LowCardinality(String) +0 \N Nullable(String) LowCardinality(String) +1 \N l Nullable(String) LowCardinality(String) +- +1 l \N LowCardinality(String) Nullable(String) +0 \N LowCardinality(String) Nullable(String) +0 \N LowCardinality(String) Nullable(String) +1 l \N LowCardinality(String) Nullable(String) +- +0 \N Nullable(String) LowCardinality(String) +1 \N l Nullable(String) LowCardinality(String) +0 \N Nullable(String) LowCardinality(String) +1 \N l Nullable(String) LowCardinality(String) +- +1 l \N LowCardinality(String) Nullable(String) +2 \N LowCardinality(String) Nullable(String) +1 l \N LowCardinality(String) Nullable(String) +2 \N LowCardinality(String) Nullable(String) +- +\N \N Nullable(String) LowCardinality(String) +1 \N l Nullable(String) LowCardinality(String) +1 \N l Nullable(String) LowCardinality(String) +\N \N Nullable(String) LowCardinality(String) +- +1 l \N LowCardinality(String) Nullable(String) +\N \N LowCardinality(String) Nullable(String) +1 l \N LowCardinality(String) Nullable(String) +\N \N LowCardinality(String) Nullable(String) +- diff --git a/tests/queries/0_stateless/01477_lc_in_merge_join_left_key.sql b/tests/queries/0_stateless/01477_lc_in_merge_join_left_key.sql new file mode 100644 index 00000000000..2507613f051 --- /dev/null +++ b/tests/queries/0_stateless/01477_lc_in_merge_join_left_key.sql @@ -0,0 +1,65 @@ +SET join_algorithm = 'auto'; +SET max_bytes_in_join = 100; + +DROP TABLE IF EXISTS t; +DROP TABLE IF EXISTS nr; + +CREATE TABLE t (`x` UInt32, `s` LowCardinality(String)) ENGINE = Memory; +CREATE TABLE nr (`x` Nullable(UInt32), `s` Nullable(String)) ENGINE = Memory; + +INSERT INTO t VALUES (1, 'l'); +INSERT INTO nr VALUES (2, NULL); + +SET join_use_nulls = 0; + +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l LEFT JOIN nr AS r USING (x) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l RIGHT JOIN nr AS r USING (x) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l FULL JOIN nr AS r USING (x) ORDER BY t.x; + +SELECT '-'; + +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l LEFT JOIN t AS r USING (x) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l RIGHT JOIN t AS r USING (x) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l FULL JOIN t AS r USING (x) ORDER BY t.x; + +SELECT '-'; + +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l LEFT JOIN nr AS r USING (s) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l RIGHT JOIN nr AS r USING (s) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l FULL JOIN nr AS r USING (s) ORDER BY t.x; + +SELECT '-'; + +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l LEFT JOIN t AS r USING (s) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l RIGHT JOIN t AS r USING (s) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l FULL JOIN t AS r USING (s) ORDER BY t.x; + +SET join_use_nulls = 1; + +SELECT '-'; + +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l LEFT JOIN nr AS r USING (x) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l RIGHT JOIN nr AS r USING (x) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l FULL JOIN nr AS r USING (x) ORDER BY t.x; + +SELECT '-'; + +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l LEFT JOIN t AS r USING (x) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l RIGHT JOIN t AS r USING (x) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l FULL JOIN t AS r USING (x) ORDER BY t.x; + +SELECT '-'; + +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l LEFT JOIN nr AS r USING (s) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l RIGHT JOIN nr AS r USING (s) ORDER BY t.x; +SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l FULL JOIN nr AS r USING (s) ORDER BY t.x; + +SELECT '-'; + +-- TODO +-- SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l LEFT JOIN t AS r USING (s) ORDER BY t.x; +-- SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l RIGHT JOIN t AS r USING (s) ORDER BY t.x; +-- SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l FULL JOIN t AS r USING (s) ORDER BY t.x; + +DROP TABLE t; +DROP TABLE nr; From ba12e68d70fada79a2a3f503defe4fe5c8144fe8 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Tue, 22 Sep 2020 13:23:10 +0300 Subject: [PATCH 271/273] Update insert_values_with_expressions.xml --- tests/performance/insert_values_with_expressions.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/performance/insert_values_with_expressions.xml b/tests/performance/insert_values_with_expressions.xml index 3456cd0ec68..daa3488e34b 100644 --- a/tests/performance/insert_values_with_expressions.xml +++ b/tests/performance/insert_values_with_expressions.xml @@ -17,7 +17,7 @@ file('test_some_expr_matches.values', Values, 'i Int64, ari Array(Int64), ars Array(String)') - select * from file('test_all_expr_matches.values', Values, 'd DateTime, i UInt32, s String, ni Nullable(UInt64), ns Nullable(String), ars Array(String)') - select * from file('test_some_expr_matches.values', Values, 'i Int64, ari Array(Int64), ars Array(String)') + select * from file('test_all_expr_matches.values', Values, 'd DateTime, i UInt32, s String, ni Nullable(UInt64), ns Nullable(String), ars Array(String)') format Null + select * from file('test_some_expr_matches.values', Values, 'i Int64, ari Array(Int64), ars Array(String)') format Null From e646aa306f8356ab91fe6b3b2df91dab4a5863e6 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Tue, 22 Sep 2020 13:25:16 +0300 Subject: [PATCH 272/273] Update columns_hashing.xml --- tests/performance/columns_hashing.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/columns_hashing.xml b/tests/performance/columns_hashing.xml index c45e5d1fe39..3ea2e013acc 100644 --- a/tests/performance/columns_hashing.xml +++ b/tests/performance/columns_hashing.xml @@ -4,7 +4,7 @@ hits_100m_single - select sum(UserID + 1 in (select UserID from hits_100m_single)) from hits_100m_single + select sum(UserID + 1 in (select UserID from hits_10m_single)) from hits_10m_single select sum((UserID + 1, RegionID) in (select UserID, RegionID from hits_10m_single)) from hits_10m_single select sum(URL in (select URL from hits_10m_single where URL != '')) from hits_10m_single select sum(MobilePhoneModel in (select MobilePhoneModel from hits_100m_single where MobilePhoneModel != '')) from hits_100m_single From e97c9b16a7105841a31aaf31fa6b5398c7cf6443 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 22 Sep 2020 13:35:28 +0300 Subject: [PATCH 273/273] more diagnostics in fuzzer --- programs/client/Client.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 7c6d386ba05..d900eb17d78 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1167,6 +1167,9 @@ private: dump_of_cloned_ast.str().c_str()); fprintf(stderr, "dump after fuzz:\n"); fuzz_base->dumpTree(std::cerr); + + fmt::print(stderr, "IAST::clone() is broken for some AST node. This is a bug. The original AST ('dump before fuzz') and its cloned copy ('dump of cloned AST') refer to the same nodes, which must never happen. This means that their parent node doesn't implement clone() correctly."); + assert(false); }