From e44e1ad0d4cb7b08e4b1de3cf863f060e4d493c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jun 2020 21:57:38 +0300 Subject: [PATCH 001/625] 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/625] 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/625] 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/625] Enable compact parts by default for small parts --- src/Interpreters/SystemLog.cpp | 3 +-- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index b432cd8803b..c2e82646de9 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -56,8 +56,7 @@ std::shared_ptr createSystemLog( else { String partition_by = config.getString(config_prefix + ".partition_by", "toYYYYMM(event_date)"); - engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time)" - "SETTINGS min_bytes_for_wide_part = '10M'"; /// Use polymorphic parts for log tables by default + engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time)"; } size_t flush_interval_milliseconds = config.getUInt64(config_prefix + ".flush_interval_milliseconds", DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index f2d2a7cc3d4..d88fe26454c 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -29,7 +29,7 @@ struct MergeTreeSettings : public SettingsCollection M(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \ \ /** Data storing format settings. */ \ - M(SettingUInt64, min_bytes_for_wide_part, 0, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ + M(SettingUInt64, min_bytes_for_wide_part, 10485760, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ M(SettingUInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \ \ /** Merge settings. */ \ From f8f67a788e4c8dc41b59d6f22631172fb4a431df Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 25 Jun 2020 19:55:45 +0300 Subject: [PATCH 005/625] allow to turn on fsync on inserts and merges --- src/Disks/DiskLocal.cpp | 17 +++++++- src/Disks/DiskLocal.h | 2 + src/Disks/DiskMemory.cpp | 5 +++ src/Disks/DiskMemory.h | 2 + src/Disks/IDisk.h | 3 ++ src/Disks/S3/DiskS3.cpp | 5 +++ src/Disks/S3/DiskS3.h | 2 + .../MergeTree/IMergeTreeDataPartWriter.cpp | 11 +++-- .../MergeTree/IMergeTreeDataPartWriter.h | 6 +-- .../MergeTree/MergeTreeDataMergerMutator.cpp | 41 +++++++++++++------ .../MergeTree/MergeTreeDataMergerMutator.h | 6 ++- .../MergeTreeDataPartWriterCompact.cpp | 4 +- .../MergeTreeDataPartWriterCompact.h | 2 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 4 +- .../MergeTree/MergeTreeDataPartWriterWide.h | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 7 +++- src/Storages/MergeTree/MergeTreeSettings.h | 3 ++ .../MergeTree/MergedBlockOutputStream.cpp | 7 ++-- .../MergeTree/MergedBlockOutputStream.h | 1 + .../MergedColumnOnlyOutputStream.cpp | 9 ++-- .../MergeTree/MergedColumnOnlyOutputStream.h | 2 +- 21 files changed, 108 insertions(+), 33 deletions(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 68f5ee99a7a..c67bac7ffe2 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -8,7 +8,7 @@ #include #include - +#include namespace DB { @@ -19,6 +19,9 @@ namespace ErrorCodes extern const int EXCESSIVE_ELEMENT_IN_CONFIG; extern const int PATH_ACCESS_DENIED; extern const int INCORRECT_DISK_INDEX; + extern const int FILE_DOESNT_EXIST; + extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_FSYNC; } std::mutex DiskLocal::reservation_mutex; @@ -188,6 +191,18 @@ void DiskLocal::moveDirectory(const String & from_path, const String & to_path) Poco::File(disk_path + from_path).renameTo(disk_path + to_path); } +void DiskLocal::sync(const String & path) const +{ + String full_path = disk_path + path; + int fd = ::open(full_path.c_str(), O_RDONLY); + if (-1 == fd) + throwFromErrnoWithPath("Cannot open file " + full_path, full_path, + errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE); + + if (-1 == fsync(fd)) + throwFromErrnoWithPath("Cannot fsync " + full_path, full_path, ErrorCodes::CANNOT_FSYNC); +} + DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) { return std::make_unique(disk_path, path); diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 61a3994b655..743ba2ceb10 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -59,6 +59,8 @@ public: void moveDirectory(const String & from_path, const String & to_path) override; + void sync(const String & path) const override; + DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void createFile(const String & path) override; diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index 3e43d159ba5..5b3350e40f7 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -261,6 +261,11 @@ void DiskMemory::moveDirectory(const String & /*from_path*/, const String & /*to throw Exception("Method moveDirectory is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); } +void DiskMemory::sync(const String & /*path*/) const +{ + throw Exception("Method sync is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); +} + DiskDirectoryIteratorPtr DiskMemory::iterateDirectory(const String & path) { std::lock_guard lock(mutex); diff --git a/src/Disks/DiskMemory.h b/src/Disks/DiskMemory.h index b0c1d30c61d..8a3ddf05aa7 100644 --- a/src/Disks/DiskMemory.h +++ b/src/Disks/DiskMemory.h @@ -52,6 +52,8 @@ public: void moveDirectory(const String & from_path, const String & to_path) override; + void sync(const String & path) const override; + DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void createFile(const String & path) override; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 011c75402f4..8de77a560d1 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -105,6 +105,9 @@ public: /// Move directory from `from_path` to `to_path`. virtual void moveDirectory(const String & from_path, const String & to_path) = 0; + /// Do fsync on directory. + virtual void sync(const String & path) const = 0; + /// Return iterator to the contents of the specified directory. virtual DiskDirectoryIteratorPtr iterateDirectory(const String & path) = 0; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 71b5991f770..292f6567df4 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -466,6 +466,11 @@ void DiskS3::clearDirectory(const String & path) remove(it->path()); } +void DiskS3::sync(const String & /*path*/) const +{ + throw Exception("Method sync is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); +} + void DiskS3::moveFile(const String & from_path, const String & to_path) { if (exists(to_path)) diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 5fa8e8358a6..09132367ae8 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -58,6 +58,8 @@ public: void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); } + void sync(const String & path) const override; + DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void moveFile(const String & from_path, const String & to_path) override; diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index 73ac7fc0064..03ae2166504 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -308,7 +308,8 @@ void IMergeTreeDataPartWriter::calculateAndSerializeSkipIndices( skip_index_data_mark = skip_index_current_data_mark; } -void IMergeTreeDataPartWriter::finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums) +void IMergeTreeDataPartWriter::finishPrimaryIndexSerialization( + MergeTreeData::DataPart::Checksums & checksums, bool sync) { bool write_final_mark = (with_final_mark && data_written); if (write_final_mark && compute_granularity) @@ -330,12 +331,14 @@ void IMergeTreeDataPartWriter::finishPrimaryIndexSerialization(MergeTreeData::Da index_stream->next(); checksums.files["primary.idx"].file_size = index_stream->count(); checksums.files["primary.idx"].file_hash = index_stream->getHash(); - index_stream = nullptr; + if (sync) + index_stream->sync(); + index_stream.reset(); } } void IMergeTreeDataPartWriter::finishSkipIndicesSerialization( - MergeTreeData::DataPart::Checksums & checksums) + MergeTreeData::DataPart::Checksums & checksums, bool sync) { for (size_t i = 0; i < skip_indices.size(); ++i) { @@ -348,6 +351,8 @@ void IMergeTreeDataPartWriter::finishSkipIndicesSerialization( { stream->finalize(); stream->addToChecksums(checksums); + if (sync) + stream->sync(); } skip_indices_streams.clear(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 2f849e7c895..eebdb880a66 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -102,9 +102,9 @@ public: void initSkipIndices(); void initPrimaryIndex(); - virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) = 0; - void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums); - void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksums); + virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) = 0; + void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums, bool sync); + void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksum, bool sync); protected: /// Count index_granularity for block and store in `index_granularity` diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 00830dd78c2..ccd7f234925 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -576,6 +576,13 @@ public: } }; +static bool needSyncPart(const size_t input_rows, size_t input_bytes, const MergeTreeSettings & settings) +{ + return ((settings.min_rows_to_sync_after_merge && input_rows >= settings.min_rows_to_sync_after_merge) + || (settings.min_compressed_bytes_to_sync_after_merge && input_bytes >= settings.min_compressed_bytes_to_sync_after_merge)); +} + + /// parts should be sorted. MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( const FutureMergedMutatedPart & future_part, @@ -648,6 +655,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor } size_t sum_input_rows_upper_bound = merge_entry->total_rows_count; + size_t sum_compressed_bytes_upper_bound = merge_entry->total_size_bytes_compressed; MergeAlgorithm merge_alg = chooseMergeAlgorithm(parts, sum_input_rows_upper_bound, gathering_columns, deduplicate, need_remove_expired_values); LOG_DEBUG(log, "Selected MergeAlgorithm: {}", ((merge_alg == MergeAlgorithm::Vertical) ? "Vertical" : "Horizontal")); @@ -803,7 +811,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (need_remove_expired_values) merged_stream = std::make_shared(merged_stream, data, metadata_snapshot, new_data_part, time_of_merge, force_ttl); - if (metadata_snapshot->hasSecondaryIndices()) { const auto & indices = metadata_snapshot->getSecondaryIndices(); @@ -863,6 +870,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (need_remove_expired_values && ttl_merges_blocker.isCancelled()) throw Exception("Cancelled merging parts with expired TTL", ErrorCodes::ABORTED); + bool need_sync = needSyncPart(sum_input_rows_upper_bound, sum_compressed_bytes_upper_bound, *data_settings); MergeTreeData::DataPart::Checksums checksums_gathered_columns; /// Gather ordinary columns @@ -942,7 +950,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); column_gathered_stream.readSuffix(); - auto changed_checksums = column_to.writeSuffixAndGetChecksums(new_data_part, checksums_gathered_columns); + auto changed_checksums = column_to.writeSuffixAndGetChecksums(new_data_part, checksums_gathered_columns, need_sync); checksums_gathered_columns.add(std::move(changed_checksums)); if (rows_written != column_elems_written) @@ -979,9 +987,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor } if (merge_alg != MergeAlgorithm::Vertical) - to.writeSuffixAndFinalizePart(new_data_part); + to.writeSuffixAndFinalizePart(new_data_part, need_sync); else - to.writeSuffixAndFinalizePart(new_data_part, &storage_columns, &checksums_gathered_columns); + to.writeSuffixAndFinalizePart(new_data_part, need_sync, &storage_columns, &checksums_gathered_columns); + + if (need_sync) + new_data_part->volume->getDisk()->sync(new_part_tmp_path); return new_data_part; } @@ -1081,7 +1092,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor /// Don't change granularity type while mutating subset of columns auto mrk_extension = source_part->index_granularity_info.is_adaptive ? getAdaptiveMrkExtension(new_data_part->getType()) : getNonAdaptiveMrkExtension(); - + bool need_sync = needSyncPart(source_part->rows_count, source_part->getBytesOnDisk(), *data_settings); bool need_remove_expired_values = false; if (in && shouldExecuteTTL(metadata_snapshot, in->getHeader().getNamesAndTypesList().getNames(), commands_for_part)) @@ -1099,7 +1110,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor time_of_mutation, compression_codec, merge_entry, - need_remove_expired_values); + need_remove_expired_values, + need_sync); /// no finalization required, because mutateAllPartColumns use /// MergedBlockOutputStream which finilaze all part fields itself @@ -1154,7 +1166,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor time_of_mutation, compression_codec, merge_entry, - need_remove_expired_values); + need_remove_expired_values, + need_sync); } for (const auto & [rename_from, rename_to] : files_to_rename) @@ -1174,6 +1187,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor finalizeMutatedPart(source_part, new_data_part, need_remove_expired_values); } + if (need_sync) + new_data_part->volume->getDisk()->sync(new_part_tmp_path); + return new_data_part; } @@ -1599,7 +1615,8 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( time_t time_of_mutation, const CompressionCodecPtr & compression_codec, MergeListEntry & merge_entry, - bool need_remove_expired_values) const + bool need_remove_expired_values, + bool need_sync) const { if (mutating_stream == nullptr) throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR); @@ -1637,7 +1654,7 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( new_data_part->minmax_idx = std::move(minmax_idx); mutating_stream->readSuffix(); - out.writeSuffixAndFinalizePart(new_data_part); + out.writeSuffixAndFinalizePart(new_data_part, need_sync); } void MergeTreeDataMergerMutator::mutateSomePartColumns( @@ -1650,7 +1667,8 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( time_t time_of_mutation, const CompressionCodecPtr & compression_codec, MergeListEntry & merge_entry, - bool need_remove_expired_values) const + bool need_remove_expired_values, + bool need_sync) const { if (mutating_stream == nullptr) throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR); @@ -1684,10 +1702,9 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( mutating_stream->readSuffix(); - auto changed_checksums = out.writeSuffixAndGetChecksums(new_data_part, new_data_part->checksums); + auto changed_checksums = out.writeSuffixAndGetChecksums(new_data_part, new_data_part->checksums, need_sync); new_data_part->checksums.add(std::move(changed_checksums)); - } void MergeTreeDataMergerMutator::finalizeMutatedPart( diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 121cc770d51..23b8d7f681b 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -189,7 +189,8 @@ private: time_t time_of_mutation, const CompressionCodecPtr & codec, MergeListEntry & merge_entry, - bool need_remove_expired_values) const; + bool need_remove_expired_values, + bool need_sync) const; /// Mutate some columns of source part with mutation_stream void mutateSomePartColumns( @@ -202,7 +203,8 @@ private: time_t time_of_mutation, const CompressionCodecPtr & codec, MergeListEntry & merge_entry, - bool need_remove_expired_values) const; + bool need_remove_expired_values, + bool need_sync) const; /// Initialize and write to disk new part fields like checksums, columns, /// etc. diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index f7a3ad75cf5..79800204a3b 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -141,7 +141,7 @@ void MergeTreeDataPartWriterCompact::writeColumnSingleGranule(const ColumnWithTy column.type->serializeBinaryBulkStateSuffix(serialize_settings, state); } -void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) +void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) { if (columns_buffer.size() != 0) writeBlock(header.cloneWithColumns(columns_buffer.releaseColumns())); @@ -158,6 +158,8 @@ void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart: stream->finalize(); stream->addToChecksums(checksums); + if (sync) + stream->sync(); stream.reset(); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 8183c038c4c..dde7deafc58 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -20,7 +20,7 @@ public: void write(const Block & block, const IColumn::Permutation * permutation, const Block & primary_key_block, const Block & skip_indexes_block) override; - void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override; + void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) override; protected: void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index e71ea4d4b94..fcd0249b10c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -264,7 +264,7 @@ void MergeTreeDataPartWriterWide::writeColumn( next_index_offset = current_row - total_rows; } -void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) +void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) { const auto & global_settings = storage.global_context.getSettingsRef(); IDataType::SerializeBinaryBulkSettings serialize_settings; @@ -295,6 +295,8 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Ch { stream.second->finalize(); stream.second->addToChecksums(checksums); + if (sync) + stream.second->sync(); } column_streams.clear(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index f5a9d17f63c..4286065a3ca 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -23,7 +23,7 @@ public: void write(const Block & block, const IColumn::Permutation * permutation, const Block & primary_key_block, const Block & skip_indexes_block) override; - void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override; + void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) override; IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenOffsetColumns & offset_columns); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 099480aca2f..cf8860b7f04 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -303,10 +303,15 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); + bool sync_on_insert = data.getSettings()->sync_after_insert; out.writePrefix(); out.writeWithPermutation(block, perm_ptr); - out.writeSuffixAndFinalizePart(new_data_part); + out.writeSuffixAndFinalizePart(new_data_part, sync_on_insert); + + /// Sync part directory. + if (sync_on_insert) + new_data_part->volume->getDisk()->sync(full_path); ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterRows, block.rows()); ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterUncompressedBytes, block.bytes()); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index f2d2a7cc3d4..da2c9ee49ee 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -43,6 +43,9 @@ struct MergeTreeSettings : public SettingsCollection M(SettingSeconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ M(SettingSeconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ M(SettingSeconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ + M(SettingUInt64, min_rows_to_sync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingUInt64, min_compressed_bytes_to_sync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingBool, sync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ \ /** Inserts settings. */ \ M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index e776a35f21f..5e15084aa7d 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -85,6 +85,7 @@ void MergedBlockOutputStream::writeSuffix() void MergedBlockOutputStream::writeSuffixAndFinalizePart( MergeTreeData::MutableDataPartPtr & new_part, + bool sync, const NamesAndTypesList * total_columns_list, MergeTreeData::DataPart::Checksums * additional_column_checksums) { @@ -95,9 +96,9 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( checksums = std::move(*additional_column_checksums); /// Finish columns serialization. - writer->finishDataSerialization(checksums); - writer->finishPrimaryIndexSerialization(checksums); - writer->finishSkipIndicesSerialization(checksums); + writer->finishDataSerialization(checksums, sync); + writer->finishPrimaryIndexSerialization(checksums, sync); + writer->finishSkipIndicesSerialization(checksums, sync); NamesAndTypesList part_columns; if (!total_columns_list) diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 1a8bf9da822..002ef78a9af 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -46,6 +46,7 @@ public: /// Finilize writing part and fill inner structures void writeSuffixAndFinalizePart( MergeTreeData::MutableDataPartPtr & new_part, + bool sync = false, const NamesAndTypesList * total_columns_list = nullptr, MergeTreeData::DataPart::Checksums * additional_column_checksums = nullptr); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 1faadd0d720..e767fb3f155 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -63,12 +63,15 @@ void MergedColumnOnlyOutputStream::writeSuffix() } MergeTreeData::DataPart::Checksums -MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums) +MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums( + MergeTreeData::MutableDataPartPtr & new_part, + MergeTreeData::DataPart::Checksums & all_checksums, + bool sync) { /// Finish columns serialization. MergeTreeData::DataPart::Checksums checksums; - writer->finishDataSerialization(checksums); - writer->finishSkipIndicesSerialization(checksums); + writer->finishDataSerialization(checksums, sync); + writer->finishSkipIndicesSerialization(checksums, sync); auto columns = new_part->getColumns(); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index 902138ced9d..507a964ede0 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -27,7 +27,7 @@ public: void write(const Block & block) override; void writeSuffix() override; MergeTreeData::DataPart::Checksums - writeSuffixAndGetChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums); + writeSuffixAndGetChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums, bool sync = false); private: Block header; From b2aa565a37076230af2ceaa32ee21fa351d37931 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 27 Jun 2020 00:55:48 +0300 Subject: [PATCH 006/625] allow to turn on fsync on inserts, merges and fetches --- src/Common/FileSyncGuard.h | 41 +++++++++++++++++++ src/Disks/DiskLocal.cpp | 35 ++++++++++------ src/Disks/DiskLocal.h | 6 ++- src/Disks/DiskMemory.cpp | 20 ++++++--- src/Disks/DiskMemory.h | 6 ++- src/Disks/IDisk.h | 12 ++++-- src/Disks/S3/DiskS3.cpp | 21 +++++++--- src/Disks/S3/DiskS3.h | 6 ++- src/Storages/MergeTree/DataPartsExchange.cpp | 16 +++++++- src/Storages/MergeTree/DataPartsExchange.h | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 +++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 15 ++++--- .../MergeTree/MergeTreeDataWriter.cpp | 12 +++--- src/Storages/MergeTree/MergeTreeSettings.h | 2 + 14 files changed, 154 insertions(+), 44 deletions(-) create mode 100644 src/Common/FileSyncGuard.h diff --git a/src/Common/FileSyncGuard.h b/src/Common/FileSyncGuard.h new file mode 100644 index 00000000000..5ec9b1d0c98 --- /dev/null +++ b/src/Common/FileSyncGuard.h @@ -0,0 +1,41 @@ +#pragma once + +#include + +namespace DB +{ + +/// Helper class, that recieves file descriptor and does fsync for it in destructor. +/// It's used to keep descriptor open, while doing some operations with it, and do fsync at the end. +/// Guaranties of sequence 'close-reopen-fsync' may depend on kernel version. +/// Source: linux-fsdevel mailing-list https://marc.info/?l=linux-fsdevel&m=152535409207496 +class FileSyncGuard +{ +public: + /// NOTE: If you have already opened descriptor, it's preffered to use + /// this constructor instead of construnctor with path. + FileSyncGuard(const DiskPtr & disk_, int fd_) : disk(disk_), fd(fd_) {} + + FileSyncGuard(const DiskPtr & disk_, const String & path) + : disk(disk_), fd(disk_->open(path, O_RDONLY)) {} + + ~FileSyncGuard() + { + try + { + disk->sync(fd); + disk->close(fd); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + +private: + DiskPtr disk; + int fd = -1; +}; + +} + diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index c67bac7ffe2..f85b69baf5e 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes extern const int FILE_DOESNT_EXIST; extern const int CANNOT_OPEN_FILE; extern const int CANNOT_FSYNC; + extern const int CANNOT_CLOSE_FILE; } std::mutex DiskLocal::reservation_mutex; @@ -191,18 +192,6 @@ void DiskLocal::moveDirectory(const String & from_path, const String & to_path) Poco::File(disk_path + from_path).renameTo(disk_path + to_path); } -void DiskLocal::sync(const String & path) const -{ - String full_path = disk_path + path; - int fd = ::open(full_path.c_str(), O_RDONLY); - if (-1 == fd) - throwFromErrnoWithPath("Cannot open file " + full_path, full_path, - errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE); - - if (-1 == fsync(fd)) - throwFromErrnoWithPath("Cannot fsync " + full_path, full_path, ErrorCodes::CANNOT_FSYNC); -} - DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) { return std::make_unique(disk_path, path); @@ -299,6 +288,28 @@ void DiskLocal::copy(const String & from_path, const std::shared_ptr & to IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } +int DiskLocal::open(const String & path, mode_t mode) const +{ + String full_path = disk_path + path; + int fd = ::open(full_path.c_str(), mode); + if (-1 == fd) + throwFromErrnoWithPath("Cannot open file " + full_path, full_path, + errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE); + return fd; +} + +void DiskLocal::close(int fd) const +{ + if (-1 == ::close(fd)) + throw Exception("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE); +} + +void DiskLocal::sync(int fd) const +{ + if (-1 == ::fsync(fd)) + throw Exception("Cannot fsync", ErrorCodes::CANNOT_FSYNC); +} + DiskPtr DiskLocalReservation::getDisk(size_t i) const { if (i != 0) diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 743ba2ceb10..d70ac06c18b 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -59,8 +59,6 @@ public: void moveDirectory(const String & from_path, const String & to_path) override; - void sync(const String & path) const override; - DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void createFile(const String & path) override; @@ -101,6 +99,10 @@ public: void createHardLink(const String & src_path, const String & dst_path) override; + int open(const String & path, mode_t mode) const override; + void close(int fd) const override; + void sync(int fd) const override; + private: bool tryReserve(UInt64 bytes); diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index 5b3350e40f7..a7f1df04e1f 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -261,11 +261,6 @@ void DiskMemory::moveDirectory(const String & /*from_path*/, const String & /*to throw Exception("Method moveDirectory is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); } -void DiskMemory::sync(const String & /*path*/) const -{ - throw Exception("Method sync is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); -} - DiskDirectoryIteratorPtr DiskMemory::iterateDirectory(const String & path) { std::lock_guard lock(mutex); @@ -413,6 +408,21 @@ void DiskMemory::setReadOnly(const String &) throw Exception("Method setReadOnly is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); } +int DiskMemory::open(const String & /*path*/, mode_t /*mode*/) const +{ + throw Exception("Method open is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); +} + +void DiskMemory::close(int /*fd*/) const +{ + throw Exception("Method close is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); +} + +void DiskMemory::sync(int /*fd*/) const +{ + throw Exception("Method sync is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); +} + using DiskMemoryPtr = std::shared_ptr; diff --git a/src/Disks/DiskMemory.h b/src/Disks/DiskMemory.h index 8a3ddf05aa7..7f111fe5e7d 100644 --- a/src/Disks/DiskMemory.h +++ b/src/Disks/DiskMemory.h @@ -52,8 +52,6 @@ public: void moveDirectory(const String & from_path, const String & to_path) override; - void sync(const String & path) const override; - DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void createFile(const String & path) override; @@ -92,6 +90,10 @@ public: void createHardLink(const String & src_path, const String & dst_path) override; + int open(const String & path, mode_t mode) const override; + void close(int fd) const override; + void sync(int fd) const override; + private: void createDirectoriesImpl(const String & path); void replaceFileImpl(const String & from_path, const String & to_path); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 8de77a560d1..bc5c9381643 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -105,9 +105,6 @@ public: /// Move directory from `from_path` to `to_path`. virtual void moveDirectory(const String & from_path, const String & to_path) = 0; - /// Do fsync on directory. - virtual void sync(const String & path) const = 0; - /// Return iterator to the contents of the specified directory. virtual DiskDirectoryIteratorPtr iterateDirectory(const String & path) = 0; @@ -174,6 +171,15 @@ public: /// Create hardlink from `src_path` to `dst_path`. virtual void createHardLink(const String & src_path, const String & dst_path) = 0; + + /// Wrapper for POSIX open + virtual int open(const String & path, mode_t mode) const = 0; + + /// Wrapper for POSIX close + virtual void close(int fd) const = 0; + + /// Wrapper for POSIX fsync + virtual void sync(int fd) const = 0; }; using DiskPtr = std::shared_ptr; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 292f6567df4..3e0fb05ed6f 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -29,6 +29,7 @@ namespace ErrorCodes extern const int CANNOT_SEEK_THROUGH_FILE; extern const int UNKNOWN_FORMAT; extern const int INCORRECT_DISK_INDEX; + extern const int NOT_IMPLEMENTED; } namespace @@ -466,11 +467,6 @@ void DiskS3::clearDirectory(const String & path) remove(it->path()); } -void DiskS3::sync(const String & /*path*/) const -{ - throw Exception("Method sync is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); -} - void DiskS3::moveFile(const String & from_path, const String & to_path) { if (exists(to_path)) @@ -669,6 +665,21 @@ void DiskS3::setReadOnly(const String & path) Poco::File(metadata_path + path).setReadOnly(true); } +int DiskS3::open(const String & /*path*/, mode_t /*mode*/) const +{ + throw Exception("Method open is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); +} + +void DiskS3::close(int /*fd*/) const +{ + throw Exception("Method close is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); +} + +void DiskS3::sync(int /*fd*/) const +{ + throw Exception("Method sync is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); +} + DiskS3Reservation::~DiskS3Reservation() { try diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 09132367ae8..cbf161da561 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -58,8 +58,6 @@ public: void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); } - void sync(const String & path) const override; - DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void moveFile(const String & from_path, const String & to_path) override; @@ -98,6 +96,10 @@ public: void setReadOnly(const String & path) override; + int open(const String & path, mode_t mode) const override; + void close(int fd) const override; + void sync(int fd) const override; + private: bool tryReserve(UInt64 bytes); diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 6796e630ff2..e7bb8206cd9 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -224,9 +225,9 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( int server_protocol_version = parse(in.getResponseCookie("server_protocol_version", "0")); ReservationPtr reservation; + size_t sum_files_size = 0; if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE) { - size_t sum_files_size; readBinary(sum_files_size, in); if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS) { @@ -247,7 +248,10 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( reservation = data.makeEmptyReservationOnLargestDisk(); } - return downloadPart(part_name, replica_path, to_detached, tmp_prefix_, std::move(reservation), in); + bool sync = (data_settings->min_compressed_bytes_to_sync_after_fetch + && sum_files_size >= data_settings->min_compressed_bytes_to_sync_after_fetch); + + return downloadPart(part_name, replica_path, to_detached, tmp_prefix_, sync, std::move(reservation), in); } MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( @@ -255,6 +259,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( const String & replica_path, bool to_detached, const String & tmp_prefix_, + bool sync, const ReservationPtr reservation, PooledReadWriteBufferFromHTTP & in) { @@ -276,6 +281,10 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( disk->createDirectories(part_download_path); + std::optional sync_guard; + if (data.getSettings()->sync_part_directory) + sync_guard.emplace(disk, part_download_path); + MergeTreeData::DataPart::Checksums checksums; for (size_t i = 0; i < files; ++i) { @@ -316,6 +325,9 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( if (file_name != "checksums.txt" && file_name != "columns.txt") checksums.addFile(file_name, file_size, expected_hash); + + if (sync) + hashing_out.sync(); } assertEOF(in); diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index c1aff6bdba5..e983d6deecf 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -71,6 +71,7 @@ private: const String & replica_path, bool to_detached, const String & tmp_prefix_, + bool sync, const ReservationPtr reservation, PooledReadWriteBufferFromHTTP & in); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 61dfeed6b7c..ab9bb7879aa 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -664,6 +665,10 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ String from = getFullRelativePath(); String to = storage.relative_data_path + new_relative_path + "/"; + std::optional sync_guard; + if (storage.getSettings()->sync_part_directory) + sync_guard.emplace(volume->getDisk(), to); + if (!volume->getDisk()->exists(from)) throw Exception("Part directory " + fullPath(volume->getDisk(), from) + " doesn't exist. Most likely it is logical error.", ErrorCodes::FILE_DOESNT_EXIST); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index ccd7f234925..9c8c4e3c1d5 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -695,6 +696,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor gathering_column_names.clear(); } + std::optional sync_guard; + if (data.getSettings()->sync_part_directory) + sync_guard.emplace(disk, new_part_tmp_path); + /** Read from all parts, merge and write into a new one. * In passing, we calculate expression for sorting. */ @@ -991,9 +996,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor else to.writeSuffixAndFinalizePart(new_data_part, need_sync, &storage_columns, &checksums_gathered_columns); - if (need_sync) - new_data_part->volume->getDisk()->sync(new_part_tmp_path); - return new_data_part; } @@ -1089,6 +1091,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor disk->createDirectories(new_part_tmp_path); + std::optional sync_guard; + if (data.getSettings()->sync_part_directory) + sync_guard.emplace(disk, new_part_tmp_path); + /// Don't change granularity type while mutating subset of columns auto mrk_extension = source_part->index_granularity_info.is_adaptive ? getAdaptiveMrkExtension(new_data_part->getType()) : getNonAdaptiveMrkExtension(); @@ -1187,9 +1193,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor finalizeMutatedPart(source_part, new_data_part, need_remove_expired_values); } - if (need_sync) - new_data_part->volume->getDisk()->sync(new_part_tmp_path); - return new_data_part; } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index cf8860b7f04..01f0b086cea 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace ProfileEvents @@ -259,7 +260,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa new_data_part->volume->getDisk()->removeRecursive(full_path); } - new_data_part->volume->getDisk()->createDirectories(full_path); + const auto disk = new_data_part->volume->getDisk(); + disk->createDirectories(full_path); + + std::optional sync_guard; + if (data.getSettings()->sync_part_directory) + sync_guard.emplace(disk, full_path); /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) @@ -309,10 +315,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa out.writeWithPermutation(block, perm_ptr); out.writeSuffixAndFinalizePart(new_data_part, sync_on_insert); - /// Sync part directory. - if (sync_on_insert) - new_data_part->volume->getDisk()->sync(full_path); - ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterRows, block.rows()); ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterUncompressedBytes, block.bytes()); ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterCompressedBytes, new_data_part->getBytesOnDisk()); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index da2c9ee49ee..c559ce2804e 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -45,7 +45,9 @@ struct MergeTreeSettings : public SettingsCollection M(SettingSeconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ M(SettingUInt64, min_rows_to_sync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ M(SettingUInt64, min_compressed_bytes_to_sync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingUInt64, min_compressed_bytes_to_sync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ M(SettingBool, sync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ + M(SettingBool, sync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ \ /** Inserts settings. */ \ M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ From c5f46b37e6a044c3a690678c7542f749a265a546 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jun 2020 00:18:27 +0300 Subject: [PATCH 007/625] Update some tests but not all --- ...system_columns_and_system_tables.reference | 2 +- .../00804_test_delta_codec_compression.sql | 12 ++++----- ...ndex_granularity_collapsing_merge_tree.sql | 18 +++++++------ ..._adaptive_index_granularity_merge_tree.sql | 25 +++++++++++-------- ...index_granularity_replacing_merge_tree.sql | 10 ++++---- ...larity_versioned_collapsing_merge_tree.sql | 6 ++--- ...test_fix_extra_seek_on_compressed_cache.sh | 2 +- .../queries/0_stateless/00933_ttl_simple.sql | 10 +------- ...hecksums_in_system_parts_columns_table.sql | 12 ++++----- .../0_stateless/01039_mergetree_exec_time.sql | 2 +- ...1042_check_query_and_last_granule_size.sql | 4 +-- .../01045_order_by_pk_special_storages.sh | 12 ++++----- .../0_stateless/01055_compact_parts.sql | 3 ++- .../00152_insert_different_granularity.sql | 4 +-- 14 files changed, 61 insertions(+), 61 deletions(-) diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference b/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference index 1d1177ba447..ff02b0ba702 100644 --- a/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference +++ b/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference @@ -6,7 +6,7 @@ │ name2 │ 1 │ 0 │ 0 │ 0 │ │ name3 │ 0 │ 0 │ 0 │ 0 │ └───────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘ -231 1 +147 1 ┌─name────────────────┬─partition_key─┬─sorting_key───┬─primary_key─┬─sampling_key─┐ │ check_system_tables │ date │ date, version │ date │ │ └─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘ diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql index ad104eff92c..91bc45df63d 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql @@ -7,12 +7,12 @@ DROP TABLE IF EXISTS default_codec_synthetic; CREATE TABLE delta_codec_synthetic ( id UInt64 Codec(Delta, ZSTD(3)) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; CREATE TABLE default_codec_synthetic ( id UInt64 Codec(ZSTD(3)) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO delta_codec_synthetic SELECT number FROM system.numbers LIMIT 5000000; INSERT INTO default_codec_synthetic SELECT number FROM system.numbers LIMIT 5000000; @@ -45,12 +45,12 @@ DROP TABLE IF EXISTS default_codec_float; CREATE TABLE delta_codec_float ( id Float64 Codec(Delta, LZ4HC) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; CREATE TABLE default_codec_float ( id Float64 Codec(LZ4HC) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO delta_codec_float SELECT number FROM numbers(1547510400, 500000) WHERE number % 3 == 0 OR number % 5 == 0 OR number % 7 == 0 OR number % 11 == 0; INSERT INTO default_codec_float SELECT * from delta_codec_float; @@ -83,12 +83,12 @@ DROP TABLE IF EXISTS default_codec_string; CREATE TABLE delta_codec_string ( id Float64 Codec(Delta, LZ4) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; CREATE TABLE default_codec_string ( id Float64 Codec(LZ4) -) ENGINE MergeTree() ORDER BY tuple(); +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO delta_codec_string SELECT concat(toString(number), toString(number % 100)) FROM numbers(1547510400, 500000); INSERT INTO default_codec_string SELECT * from delta_codec_string; diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql index b610d605e23..5603b722513 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql @@ -8,10 +8,11 @@ CREATE TABLE zero_rows_per_granule ( v2 Int64, Sign Int8 ) ENGINE CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(p) ORDER BY k - SETTINGS index_granularity_bytes=20, write_final_mark = 0, - enable_vertical_merge_algorithm=1, - vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + SETTINGS index_granularity_bytes=20, write_final_mark = 0, + min_bytes_for_wide_part = 0, + enable_vertical_merge_algorithm=1, + vertical_merge_algorithm_min_rows_to_activate=0, + vertical_merge_algorithm_min_columns_to_activate=0; INSERT INTO zero_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, 1), ('2018-05-16', 2, 3000, 4000, 1), ('2018-05-17', 3, 5000, 6000, 1), ('2018-05-18', 4, 7000, 8000, 1); @@ -39,10 +40,11 @@ CREATE TABLE four_rows_per_granule ( v2 Int64, Sign Int8 ) ENGINE CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(p) ORDER BY k - SETTINGS index_granularity_bytes=110, write_final_mark = 0, - enable_vertical_merge_algorithm=1, - vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + SETTINGS index_granularity_bytes=110, write_final_mark = 0, + min_bytes_for_wide_part = 0, + enable_vertical_merge_algorithm=1, + vertical_merge_algorithm_min_rows_to_activate=0, + vertical_merge_algorithm_min_columns_to_activate=0; INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, 1), ('2018-05-16', 2, 3000, 4000, 1), ('2018-05-17', 3, 5000, 6000, 1), ('2018-05-18', 4, 7000, 8000, 1); diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.sql index 18ebebb316c..48b6fae19fe 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.sql @@ -7,7 +7,7 @@ CREATE TABLE zero_rows_per_granule ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, write_final_mark = 0; +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO zero_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -34,7 +34,7 @@ CREATE TABLE two_rows_per_granule ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 40, write_final_mark = 0; +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 40, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO two_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -61,7 +61,7 @@ CREATE TABLE four_rows_per_granule ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0; +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO four_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -95,7 +95,7 @@ CREATE TABLE huge_granularity_small_blocks ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 1000000, write_final_mark = 0; +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 1000000, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO huge_granularity_small_blocks (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -126,7 +126,7 @@ CREATE TABLE adaptive_granularity_alter ( k UInt64, v1 UInt64, v2 Int64 -) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0; +) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0, min_bytes_for_wide_part = 0; INSERT INTO adaptive_granularity_alter (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -183,7 +183,8 @@ CREATE TABLE zero_rows_per_granule ( SETTINGS index_granularity_bytes=20, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0; INSERT INTO zero_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -215,7 +216,8 @@ CREATE TABLE two_rows_per_granule ( SETTINGS index_granularity_bytes=40, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0; INSERT INTO two_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -246,7 +248,8 @@ CREATE TABLE four_rows_per_granule ( SETTINGS index_granularity_bytes = 110, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0; INSERT INTO four_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -284,7 +287,8 @@ CREATE TABLE huge_granularity_small_blocks ( SETTINGS index_granularity_bytes=1000000, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0; INSERT INTO huge_granularity_small_blocks (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -319,7 +323,8 @@ CREATE TABLE adaptive_granularity_alter ( SETTINGS index_granularity_bytes=110, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0; INSERT INTO adaptive_granularity_alter (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_replacing_merge_tree.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_replacing_merge_tree.sql index f72d5f0f9cb..53a546f9d0f 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_replacing_merge_tree.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_replacing_merge_tree.sql @@ -10,7 +10,7 @@ CREATE TABLE zero_rows_per_granule ( SETTINGS index_granularity_bytes=20, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO zero_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -41,7 +41,7 @@ CREATE TABLE two_rows_per_granule ( SETTINGS index_granularity_bytes=40, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO two_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -72,7 +72,7 @@ CREATE TABLE four_rows_per_granule ( SETTINGS index_granularity_bytes = 110, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO four_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -118,7 +118,7 @@ CREATE TABLE huge_granularity_small_blocks ( SETTINGS index_granularity_bytes=1000000, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO huge_granularity_small_blocks (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -154,7 +154,7 @@ CREATE TABLE adaptive_granularity_alter ( SETTINGS index_granularity_bytes=110, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO adaptive_granularity_alter (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.sql index c5b65839b2a..05f4dc835e5 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.sql @@ -12,7 +12,7 @@ CREATE TABLE zero_rows_per_granule ( SETTINGS index_granularity_bytes=20, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO zero_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 2, 3000, 4000, 1, 1), ('2018-05-17', 3, 5000, 6000, 1, 1), ('2018-05-18', 4, 7000, 8000, 1, 1); @@ -44,7 +44,7 @@ CREATE TABLE four_rows_per_granule ( SETTINGS index_granularity_bytes=120, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 2, 3000, 4000, 1, 1), ('2018-05-17', 3, 5000, 6000, 1, 1), ('2018-05-18', 4, 7000, 8000, 1, 1); @@ -89,7 +89,7 @@ CREATE TABLE six_rows_per_granule ( SETTINGS index_granularity_bytes=170, write_final_mark = 0, enable_vertical_merge_algorithm=1, vertical_merge_algorithm_min_rows_to_activate=0, - vertical_merge_algorithm_min_columns_to_activate=0; + vertical_merge_algorithm_min_columns_to_activate=0, min_bytes_for_wide_part = 0; INSERT INTO six_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 1, 1000, 2000, -1, 2); diff --git a/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh b/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh index 1f7571a2404..e0225f0d31d 100755 --- a/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh +++ b/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS small_table" -$CLICKHOUSE_CLIENT --query="CREATE TABLE small_table (a UInt64 default 0, n UInt64) ENGINE = MergeTree() PARTITION BY tuple() ORDER BY (a);" +$CLICKHOUSE_CLIENT --query="CREATE TABLE small_table (a UInt64 default 0, n UInt64) ENGINE = MergeTree() PARTITION BY tuple() ORDER BY (a) SETTINGS min_bytes_for_wide_part = 0;" $CLICKHOUSE_CLIENT --query="INSERT INTO small_table(n) SELECT * from system.numbers limit 100000;" diff --git a/tests/queries/0_stateless/00933_ttl_simple.sql b/tests/queries/0_stateless/00933_ttl_simple.sql index b924faad3f5..83d9962043d 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.sql +++ b/tests/queries/0_stateless/00933_ttl_simple.sql @@ -13,9 +13,8 @@ create table ttl_00933_1 (d DateTime, a Int, b Int) engine = MergeTree order by insert into ttl_00933_1 values (now(), 1, 2); insert into ttl_00933_1 values (now(), 3, 4); insert into ttl_00933_1 values (now() + 1000, 5, 6); +select sleep(1.1) format Null; optimize table ttl_00933_1 final; -- check ttl merge for part with both expired and unexpired values -select sleep(1.1) format Null; -- wait if very fast merge happen -optimize table ttl_00933_1 final; select a, b from ttl_00933_1; drop table if exists ttl_00933_1; @@ -24,7 +23,6 @@ create table ttl_00933_1 (d DateTime, a Int ttl d + interval 1 DAY) engine = Mer insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 3); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1 order by d; @@ -34,7 +32,6 @@ create table ttl_00933_1 (d DateTime, a Int) engine = MergeTree order by tuple() insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl_00933_1 values (toDateTime('2100-10-10 00:00:00'), 3); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1 order by d; @@ -43,7 +40,6 @@ drop table if exists ttl_00933_1; create table ttl_00933_1 (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) ttl d + interval 1 day; insert into ttl_00933_1 values (toDate('2000-10-10'), 1); insert into ttl_00933_1 values (toDate('2100-10-10'), 2); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1 order by d; @@ -52,7 +48,6 @@ drop table if exists ttl_00933_1; create table ttl_00933_1 (b Int, a Int ttl now()-1000) engine = MergeTree order by tuple() partition by tuple(); show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1; @@ -61,7 +56,6 @@ drop table if exists ttl_00933_1; create table ttl_00933_1 (b Int, a Int ttl now()+1000) engine = MergeTree order by tuple() partition by tuple(); show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1; @@ -70,7 +64,6 @@ drop table if exists ttl_00933_1; create table ttl_00933_1 (b Int, a Int ttl today()-1) engine = MergeTree order by tuple() partition by tuple(); show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1; @@ -79,7 +72,6 @@ drop table if exists ttl_00933_1; create table ttl_00933_1 (b Int, a Int ttl today()+1) engine = MergeTree order by tuple() partition by tuple(); show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); -select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1; diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql index b9eed1e8183..e865ed609be 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql @@ -1,14 +1,14 @@ DROP TABLE IF EXISTS test_00961; -CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree(d, (a, b), 111); +CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree(d, (a, b), 111) SETTINGS min_bytes_for_wide_part = 0; INSERT INTO test_00961 VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789); -SELECT - name, - table, - hash_of_all_files, - hash_of_uncompressed_files, +SELECT + name, + table, + hash_of_all_files, + hash_of_uncompressed_files, uncompressed_hash_of_compressed_files FROM system.parts WHERE table = 'test_00961' and database = currentDatabase(); diff --git a/tests/queries/0_stateless/01039_mergetree_exec_time.sql b/tests/queries/0_stateless/01039_mergetree_exec_time.sql index 4cefb2e9305..d3aade41cea 100644 --- a/tests/queries/0_stateless/01039_mergetree_exec_time.sql +++ b/tests/queries/0_stateless/01039_mergetree_exec_time.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS tab; -create table tab (A Int64) Engine=MergeTree order by tuple(); +create table tab (A Int64) Engine=MergeTree order by tuple() SETTINGS min_bytes_for_wide_part = 0; insert into tab select cityHash64(number) from numbers(1000); select sum(sleep(0.1)) from tab settings max_block_size = 1, max_execution_time=1; -- { serverError 159 } DROP TABLE IF EXISTS tab; diff --git a/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql b/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql index 9777ea1dc45..c62fe25a041 100644 --- a/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql +++ b/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql @@ -1,7 +1,7 @@ SET check_query_single_value_result = 0; DROP TABLE IF EXISTS check_query_test; -CREATE TABLE check_query_test (SomeKey UInt64, SomeValue String) ENGINE = MergeTree() ORDER BY SomeKey; +CREATE TABLE check_query_test (SomeKey UInt64, SomeValue String) ENGINE = MergeTree() ORDER BY SomeKey SETTINGS min_bytes_for_wide_part = 0; -- Number of rows in last granule should be equals to granularity. -- Rows in this table are short, so granularity will be 8192. @@ -17,7 +17,7 @@ DROP TABLE IF EXISTS check_query_test; DROP TABLE IF EXISTS check_query_test_non_adaptive; -CREATE TABLE check_query_test_non_adaptive (SomeKey UInt64, SomeValue String) ENGINE = MergeTree() ORDER BY SomeKey SETTINGS index_granularity_bytes = 0; +CREATE TABLE check_query_test_non_adaptive (SomeKey UInt64, SomeValue String) ENGINE = MergeTree() ORDER BY SomeKey SETTINGS index_granularity_bytes = 0, min_bytes_for_wide_part = 0; INSERT INTO check_query_test_non_adaptive SELECT number, toString(number) FROM system.numbers LIMIT 81920; diff --git a/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh b/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh index 0898fec802c..b0d63b75dd5 100755 --- a/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh +++ b/tests/queries/0_stateless/01045_order_by_pk_special_storages.sh @@ -10,11 +10,11 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS s2" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS m" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS buf" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mv" - -$CLICKHOUSE_CLIENT -q "CREATE TABLE s1 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3" -$CLICKHOUSE_CLIENT -q "CREATE TABLE s2 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3" -$CLICKHOUSE_CLIENT -q "CREATE TABLE m (a UInt32, s String) engine = Merge(currentDatabase(), 's[1,2]')" +$CLICKHOUSE_CLIENT -q "CREATE TABLE s1 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3 SETTINGS min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE s2 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3 SETTINGS min_bytes_for_wide_part = 0" + +$CLICKHOUSE_CLIENT -q "CREATE TABLE m (a UInt32, s String) engine = Merge(currentDatabase(), 's[1,2]') SETTINGS min_bytes_for_wide_part = 0" $CLICKHOUSE_CLIENT -q "INSERT INTO s1 select (number % 20) * 2 as n, toString(number * number) from numbers(100000)" $CLICKHOUSE_CLIENT -q "INSERT INTO s2 select (number % 20) * 2 + 1 as n, toString(number * number * number) from numbers(100000)" @@ -45,7 +45,7 @@ else fi $CLICKHOUSE_CLIENT -q "SELECT '---MaterializedView---'" -$CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW mv (a UInt32, s String) engine = MergeTree ORDER BY s POPULATE AS SELECT a, s FROM s1 WHERE a % 7 = 0" +$CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW mv (a UInt32, s String) engine = MergeTree ORDER BY s SETTINGS min_bytes_for_wide_part = 0 POPULATE AS SELECT a, s FROM s1 WHERE a % 7 = 0" $CLICKHOUSE_CLIENT -q "SELECT a, s FROM mv ORDER BY s LIMIT 10" rows_read=`$CLICKHOUSE_CLIENT -q "SELECT a, s FROM mv ORDER BY s LIMIT 10 FORMAT JSON" --max_threads=1 --max_block_size=20 | grep "rows_read" | sed 's/[^0-9]*//g'` @@ -59,4 +59,4 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS s1" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS s2" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS m" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS buf" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mv" \ No newline at end of file +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mv" diff --git a/tests/queries/0_stateless/01055_compact_parts.sql b/tests/queries/0_stateless/01055_compact_parts.sql index 05b0657ba7e..e99af76439c 100755 --- a/tests/queries/0_stateless/01055_compact_parts.sql +++ b/tests/queries/0_stateless/01055_compact_parts.sql @@ -3,9 +3,10 @@ set mutations_sync = 2; drop table if exists mt_compact; create table mt_compact(a UInt64, b UInt64 DEFAULT a * a, s String, n Nested(x UInt32, y String), lc LowCardinality(String)) -engine = MergeTree +engine = MergeTree order by a partition by a % 10 settings index_granularity = 8, +min_bytes_for_wide_part = 0, min_rows_for_wide_part = 10; insert into mt_compact (a, s, n.y, lc) select number, toString((number * 2132214234 + 5434543) % 2133443), ['a', 'b', 'c'], number % 2 ? 'bar' : 'baz' from numbers(90); diff --git a/tests/queries/1_stateful/00152_insert_different_granularity.sql b/tests/queries/1_stateful/00152_insert_different_granularity.sql index 5ca34bbe48e..7e04aedf2dd 100644 --- a/tests/queries/1_stateful/00152_insert_different_granularity.sql +++ b/tests/queries/1_stateful/00152_insert_different_granularity.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS fixed_granularity_table; -CREATE TABLE fixed_granularity_table (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, index_granularity_bytes=0; -- looks like default table before update +CREATE TABLE fixed_granularity_table (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, index_granularity_bytes=0, min_bytes_for_wide_part = 0; -- looks like default table before update ALTER TABLE fixed_granularity_table REPLACE PARTITION 201403 FROM test.hits; @@ -29,7 +29,7 @@ ALTER TABLE test.hits ATTACH PARTITION 201403; DROP TABLE IF EXISTS hits_copy; -CREATE TABLE hits_copy (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, index_granularity_bytes=0; +CREATE TABLE hits_copy (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, index_granularity_bytes=0, min_bytes_for_wide_part = 0; ALTER TABLE hits_copy REPLACE PARTITION 201403 FROM test.hits; From ca346ea13cd0ad0f02a29d59302584c826b52298 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 3 Jul 2020 02:41:37 +0300 Subject: [PATCH 008/625] rename fsync-related settings --- src/Storages/MergeTree/DataPartsExchange.cpp | 6 +++--- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeSettings.h | 10 +++++----- 5 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index e7bb8206cd9..72b478cf587 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -248,8 +248,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( reservation = data.makeEmptyReservationOnLargestDisk(); } - bool sync = (data_settings->min_compressed_bytes_to_sync_after_fetch - && sum_files_size >= data_settings->min_compressed_bytes_to_sync_after_fetch); + bool sync = (data_settings->min_compressed_bytes_to_fsync_after_fetch + && sum_files_size >= data_settings->min_compressed_bytes_to_fsync_after_fetch); return downloadPart(part_name, replica_path, to_detached, tmp_prefix_, sync, std::move(reservation), in); } @@ -282,7 +282,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( disk->createDirectories(part_download_path); std::optional sync_guard; - if (data.getSettings()->sync_part_directory) + if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, part_download_path); MergeTreeData::DataPart::Checksums checksums; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index ab9bb7879aa..3d8cb6b7fc5 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -666,7 +666,7 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ String to = storage.relative_data_path + new_relative_path + "/"; std::optional sync_guard; - if (storage.getSettings()->sync_part_directory) + if (storage.getSettings()->fsync_part_directory) sync_guard.emplace(volume->getDisk(), to); if (!volume->getDisk()->exists(from)) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 9c8c4e3c1d5..c39d1981031 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -579,8 +579,8 @@ public: static bool needSyncPart(const size_t input_rows, size_t input_bytes, const MergeTreeSettings & settings) { - return ((settings.min_rows_to_sync_after_merge && input_rows >= settings.min_rows_to_sync_after_merge) - || (settings.min_compressed_bytes_to_sync_after_merge && input_bytes >= settings.min_compressed_bytes_to_sync_after_merge)); + return ((settings.min_rows_to_fsync_after_merge && input_rows >= settings.min_rows_to_fsync_after_merge) + || (settings.min_compressed_bytes_to_fsync_after_merge && input_bytes >= settings.min_compressed_bytes_to_fsync_after_merge)); } @@ -697,7 +697,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor } std::optional sync_guard; - if (data.getSettings()->sync_part_directory) + if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, new_part_tmp_path); /** Read from all parts, merge and write into a new one. @@ -1092,7 +1092,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor disk->createDirectories(new_part_tmp_path); std::optional sync_guard; - if (data.getSettings()->sync_part_directory) + if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, new_part_tmp_path); /// Don't change granularity type while mutating subset of columns diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 01f0b086cea..23210fc604e 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -264,7 +264,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa disk->createDirectories(full_path); std::optional sync_guard; - if (data.getSettings()->sync_part_directory) + if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, full_path); /// If we need to calculate some columns to sort. @@ -309,7 +309,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); - bool sync_on_insert = data.getSettings()->sync_after_insert; + bool sync_on_insert = data.getSettings()->fsync_after_insert; out.writePrefix(); out.writeWithPermutation(block, perm_ptr); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index c559ce2804e..eeee0c4b1e1 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -43,11 +43,11 @@ struct MergeTreeSettings : public SettingsCollection M(SettingSeconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ M(SettingSeconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ M(SettingSeconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ - M(SettingUInt64, min_rows_to_sync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ - M(SettingUInt64, min_compressed_bytes_to_sync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ - M(SettingUInt64, min_compressed_bytes_to_sync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ - M(SettingBool, sync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ - M(SettingBool, sync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ + M(SettingUInt64, min_rows_to_fsync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingUInt64, min_compressed_bytes_to_fsync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingUInt64, min_compressed_bytes_to_fsync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ + M(SettingBool, fsync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ + M(SettingBool, fsync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ \ /** Inserts settings. */ \ M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ From 2d43519e038ded3bef0962a3d5ff7a5da7248914 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Jul 2020 02:27:20 +0300 Subject: [PATCH 009/625] 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 230938d3a3082fbf241c9d873571231a69a5f450 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 11 Jul 2020 15:12:42 +0800 Subject: [PATCH 010/625] Refactor joinGet and implement multi-key lookup. --- src/Functions/FunctionJoinGet.cpp | 83 +++++++++---------- src/Functions/FunctionJoinGet.h | 11 +-- src/Interpreters/HashJoin.cpp | 69 ++++++++------- src/Interpreters/HashJoin.h | 10 +-- src/Interpreters/misc.h | 2 +- .../0_stateless/01080_join_get_null.reference | 2 +- .../0_stateless/01080_join_get_null.sql | 12 +-- .../01400_join_get_with_multi_keys.reference | 1 + .../01400_join_get_with_multi_keys.sql | 9 ++ 9 files changed, 104 insertions(+), 95 deletions(-) create mode 100644 tests/queries/0_stateless/01400_join_get_with_multi_keys.reference create mode 100644 tests/queries/0_stateless/01400_join_get_with_multi_keys.sql diff --git a/src/Functions/FunctionJoinGet.cpp b/src/Functions/FunctionJoinGet.cpp index a33b70684a5..1badc689c6a 100644 --- a/src/Functions/FunctionJoinGet.cpp +++ b/src/Functions/FunctionJoinGet.cpp @@ -1,10 +1,10 @@ #include +#include #include #include #include #include -#include #include @@ -16,19 +16,35 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +template +void ExecutableFunctionJoinGet::execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t) +{ + Block keys; + for (size_t i = 2; i < arguments.size(); ++i) + { + auto key = block.getByPosition(arguments[i]); + keys.insert(std::move(key)); + } + block.getByPosition(result) = join->joinGet(keys, result_block); +} + +template +ExecutableFunctionImplPtr FunctionJoinGet::prepare(const Block &, const ColumnNumbers &, size_t) const +{ + return std::make_unique>(join, Block{{return_type->createColumn(), return_type, attr_name}}); +} + static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & context) { - if (arguments.size() != 3) - throw Exception{"Function joinGet takes 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - String join_name; if (const auto * name_col = checkAndGetColumnConst(arguments[0].column.get())) { join_name = name_col->getValue(); } else - throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function joinGet, expected a const string.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception( + "Illegal type " + arguments[0].type->getName() + " of first argument of function joinGet, expected a const string.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); size_t dot = join_name.find('.'); String database_name; @@ -43,10 +59,12 @@ static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & co ++dot; } String table_name = join_name.substr(dot); + if (table_name.empty()) + throw Exception("joinGet does not allow empty table name", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); auto table = DatabaseCatalog::instance().getTable({database_name, table_name}, context); auto storage_join = std::dynamic_pointer_cast(table); if (!storage_join) - throw Exception{"Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception("Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); String attr_name; if (const auto * name_col = checkAndGetColumnConst(arguments[1].column.get())) @@ -54,57 +72,30 @@ static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & co attr_name = name_col->getValue(); } else - throw Exception{"Illegal type " + arguments[1].type->getName() - + " of second argument of function joinGet, expected a const string.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception( + "Illegal type " + arguments[1].type->getName() + " of second argument of function joinGet, expected a const string.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_pair(storage_join, attr_name); } template FunctionBaseImplPtr JoinGetOverloadResolver::build(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const { + if (arguments.size() < 3) + throw Exception( + "Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + + ", should be greater or equal to 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); auto [storage_join, attr_name] = getJoin(arguments, context); auto join = storage_join->getJoin(); - DataTypes data_types(arguments.size()); - + DataTypes data_types(arguments.size() - 2); + for (size_t i = 2; i < arguments.size(); ++i) + data_types[i - 2] = arguments[i].type; + auto return_type = join->joinGetCheckAndGetReturnType(data_types, attr_name, or_null); auto table_lock = storage_join->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); - for (size_t i = 0; i < arguments.size(); ++i) - data_types[i] = arguments[i].type; - - auto return_type = join->joinGetReturnType(attr_name, or_null); return std::make_unique>(table_lock, storage_join, join, attr_name, data_types, return_type); } -template -DataTypePtr JoinGetOverloadResolver::getReturnType(const ColumnsWithTypeAndName & arguments) const -{ - auto [storage_join, attr_name] = getJoin(arguments, context); - auto join = storage_join->getJoin(); - return join->joinGetReturnType(attr_name, or_null); -} - - -template -void ExecutableFunctionJoinGet::execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - auto ctn = block.getByPosition(arguments[2]); - if (isColumnConst(*ctn.column)) - ctn.column = ctn.column->cloneResized(1); - ctn.name = ""; // make sure the key name never collide with the join columns - Block key_block = {ctn}; - join->joinGet(key_block, attr_name, or_null); - auto & result_ctn = key_block.getByPosition(1); - if (isColumnConst(*ctn.column)) - result_ctn.column = ColumnConst::create(result_ctn.column, input_rows_count); - block.getByPosition(result) = result_ctn; -} - -template -ExecutableFunctionImplPtr FunctionJoinGet::prepare(const Block &, const ColumnNumbers &, size_t) const -{ - return std::make_unique>(join, attr_name); -} - void registerFunctionJoinGet(FunctionFactory & factory) { // joinGet diff --git a/src/Functions/FunctionJoinGet.h b/src/Functions/FunctionJoinGet.h index a82da589960..6b3b1202f60 100644 --- a/src/Functions/FunctionJoinGet.h +++ b/src/Functions/FunctionJoinGet.h @@ -13,14 +13,14 @@ template class ExecutableFunctionJoinGet final : public IExecutableFunctionImpl { public: - ExecutableFunctionJoinGet(HashJoinPtr join_, String attr_name_) - : join(std::move(join_)), attr_name(std::move(attr_name_)) {} + ExecutableFunctionJoinGet(HashJoinPtr join_, const Block & result_block_) + : join(std::move(join_)), result_block(result_block_) {} static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet"; bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } bool useDefaultImplementationForLowCardinalityColumns() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; @@ -28,7 +28,7 @@ public: private: HashJoinPtr join; - const String attr_name; + Block result_block; }; template @@ -77,13 +77,14 @@ public: String getName() const override { return name; } FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const override; - DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const override; + DataTypePtr getReturnType(const ColumnsWithTypeAndName &) const override { return {}; } // Not used bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForLowCardinalityColumns() const override { return true; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } private: const Context & context; diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 27294a57675..ffc806b9e88 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -42,6 +42,7 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; extern const int SET_SIZE_LIMIT_EXCEEDED; extern const int TYPE_MISMATCH; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } namespace @@ -1109,27 +1110,34 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) block = block.cloneWithColumns(std::move(dst_columns)); } -static void checkTypeOfKey(const Block & block_left, const Block & block_right) -{ - const auto & [c1, left_type_origin, left_name] = block_left.safeGetByPosition(0); - const auto & [c2, right_type_origin, right_name] = block_right.safeGetByPosition(0); - auto left_type = removeNullable(left_type_origin); - auto right_type = removeNullable(right_type_origin); - if (!left_type->equals(*right_type)) - throw Exception("Type mismatch of columns to joinGet by: " - + left_name + " " + left_type->getName() + " at left, " - + right_name + " " + right_type->getName() + " at right", - ErrorCodes::TYPE_MISMATCH); -} - - -DataTypePtr HashJoin::joinGetReturnType(const String & column_name, bool or_null) const +DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const { std::shared_lock lock(data->rwlock); + size_t num_keys = data_types.size(); + if (right_table_keys.columns() != num_keys) + throw Exception( + "Number of arguments for function joinGet" + toString(or_null ? "OrNull" : "") + + " doesn't match: passed, should be equal to " + toString(num_keys), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (size_t i = 0; i < num_keys; ++i) + { + const auto & left_type_origin = data_types[i]; + const auto & [c2, right_type_origin, right_name] = right_table_keys.safeGetByPosition(i); + auto left_type = removeNullable(left_type_origin); + auto right_type = removeNullable(right_type_origin); + if (!left_type->equals(*right_type)) + throw Exception( + "Type mismatch in joinGet key " + toString(i) + ": found type " + left_type->getName() + ", while the needed type is " + + right_type->getName(), + ErrorCodes::TYPE_MISMATCH); + } + if (!sample_block_with_columns_to_add.has(column_name)) throw Exception("StorageJoin doesn't contain column " + column_name, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + auto elem = sample_block_with_columns_to_add.getByName(column_name); if (or_null) elem.type = makeNullable(elem.type); @@ -1138,34 +1146,33 @@ DataTypePtr HashJoin::joinGetReturnType(const String & column_name, bool or_null template -void HashJoin::joinGetImpl(Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const +ColumnWithTypeAndName HashJoin::joinGetImpl(const Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const { - joinBlockImpl( - block, {block.getByPosition(0).name}, block_with_columns_to_add, maps_); + // Assemble the key block with correct names. + Block keys; + for (size_t i = 0; i < block.columns(); ++i) + { + auto key = block.getByPosition(i); + key.name = key_names_right[i]; + keys.insert(std::move(key)); + } + + joinBlockImpl( + keys, key_names_right, block_with_columns_to_add, maps_); + return keys.getByPosition(keys.columns() - 1); } -// TODO: support composite key // TODO: return multiple columns as named tuple // TODO: return array of values when strictness == ASTTableJoin::Strictness::All -void HashJoin::joinGet(Block & block, const String & column_name, bool or_null) const +ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const { std::shared_lock lock(data->rwlock); - if (key_names_right.size() != 1) - throw Exception("joinGet only supports StorageJoin containing exactly one key", ErrorCodes::UNSUPPORTED_JOIN_KEYS); - - checkTypeOfKey(block, right_table_keys); - - auto elem = sample_block_with_columns_to_add.getByName(column_name); - if (or_null) - elem.type = makeNullable(elem.type); - elem.column = elem.type->createColumn(); - if ((strictness == ASTTableJoin::Strictness::Any || strictness == ASTTableJoin::Strictness::RightAny) && kind == ASTTableJoin::Kind::Left) { - joinGetImpl(block, {elem}, std::get(data->maps)); + return joinGetImpl(block, block_with_columns_to_add, std::get(data->maps)); } else throw Exception("joinGet only supports StorageJoin of type Left Any", ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN); diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 67d83d27a6d..025f41ac28f 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -162,11 +162,11 @@ public: */ void joinBlock(Block & block, ExtraBlockPtr & not_processed) override; - /// Infer the return type for joinGet function - DataTypePtr joinGetReturnType(const String & column_name, bool or_null) const; + /// Check joinGet arguments and infer the return type. + DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const; - /// Used by joinGet function that turns StorageJoin into a dictionary - void joinGet(Block & block, const String & column_name, bool or_null) const; + /// Used by joinGet function that turns StorageJoin into a dictionary. + ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add) const; /** Keep "totals" (separate part of dataset, see WITH TOTALS) to use later. */ @@ -383,7 +383,7 @@ private: void joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const; template - void joinGetImpl(Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const; + ColumnWithTypeAndName joinGetImpl(const Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const; static Type chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes); }; diff --git a/src/Interpreters/misc.h b/src/Interpreters/misc.h index 094dfbbbb81..cae2691ca1f 100644 --- a/src/Interpreters/misc.h +++ b/src/Interpreters/misc.h @@ -28,7 +28,7 @@ inline bool functionIsLikeOperator(const std::string & name) inline bool functionIsJoinGet(const std::string & name) { - return name == "joinGet" || startsWith(name, "dictGet"); + return startsWith(name, "joinGet"); } inline bool functionIsDictGet(const std::string & name) diff --git a/tests/queries/0_stateless/01080_join_get_null.reference b/tests/queries/0_stateless/01080_join_get_null.reference index bfde072a796..0cfbf08886f 100644 --- a/tests/queries/0_stateless/01080_join_get_null.reference +++ b/tests/queries/0_stateless/01080_join_get_null.reference @@ -1 +1 @@ -2 2 +2 diff --git a/tests/queries/0_stateless/01080_join_get_null.sql b/tests/queries/0_stateless/01080_join_get_null.sql index 71e7ddf8e75..9f782452d34 100644 --- a/tests/queries/0_stateless/01080_join_get_null.sql +++ b/tests/queries/0_stateless/01080_join_get_null.sql @@ -1,12 +1,12 @@ DROP TABLE IF EXISTS test_joinGet; -DROP TABLE IF EXISTS test_join_joinGet; -CREATE TABLE test_joinGet(id Int32, user_id Nullable(Int32)) Engine = Memory(); -CREATE TABLE test_join_joinGet(user_id Int32, name String) Engine = Join(ANY, LEFT, user_id); +CREATE TABLE test_joinGet(user_id Nullable(Int32), name String) Engine = Join(ANY, LEFT, user_id); -INSERT INTO test_join_joinGet VALUES (2, 'a'), (6, 'b'), (10, 'c'); +INSERT INTO test_joinGet VALUES (2, 'a'), (6, 'b'), (10, 'c'), (null, 'd'); -SELECT 2 id, toNullable(toInt32(2)) user_id WHERE joinGet(test_join_joinGet, 'name', user_id) != ''; +SELECT toNullable(toInt32(2)) user_id WHERE joinGet(test_joinGet, 'name', user_id) != ''; + +-- If the JOIN keys are Nullable fields, the rows where at least one of the keys has the value NULL are not joined. +SELECT cast(null AS Nullable(Int32)) user_id WHERE joinGet(test_joinGet, 'name', user_id) != ''; DROP TABLE test_joinGet; -DROP TABLE test_join_joinGet; diff --git a/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference b/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference new file mode 100644 index 00000000000..49d59571fbf --- /dev/null +++ b/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference @@ -0,0 +1 @@ +0.1 diff --git a/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql b/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql new file mode 100644 index 00000000000..73068270762 --- /dev/null +++ b/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS test_joinGet; + +CREATE TABLE test_joinGet(a String, b String, c Float64) ENGINE = Join(any, left, a, b); + +INSERT INTO test_joinGet VALUES ('ab', '1', 0.1), ('ab', '2', 0.2), ('cd', '3', 0.3); + +SELECT joinGet(test_joinGet, 'c', 'ab', '1'); + +DROP TABLE test_joinGet; From 4331158d3051437f44c7fa1271e4673272cf8cac Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Aug 2020 16:09:23 +0300 Subject: [PATCH 011/625] merge with master --- src/Disks/DiskDecorator.cpp | 15 +++++++++++++++ src/Disks/DiskDecorator.h | 3 +++ .../MergeTree/MergeTreeDataPartWriterInMemory.cpp | 2 +- .../MergeTree/MergeTreeDataPartWriterInMemory.h | 2 +- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 11 ++++++----- 5 files changed, 26 insertions(+), 7 deletions(-) diff --git a/src/Disks/DiskDecorator.cpp b/src/Disks/DiskDecorator.cpp index e55534e347f..7f2ea58d7cf 100644 --- a/src/Disks/DiskDecorator.cpp +++ b/src/Disks/DiskDecorator.cpp @@ -165,4 +165,19 @@ void DiskDecorator::truncateFile(const String & path, size_t size) delegate->truncateFile(path, size); } +int DiskDecorator::open(const String & path, mode_t mode) const +{ + return delegate->open(path, mode); +} + +void DiskDecorator::close(int fd) const +{ + delegate->close(fd); +} + +void DiskDecorator::sync(int fd) const +{ + delegate->sync(fd); +} + } diff --git a/src/Disks/DiskDecorator.h b/src/Disks/DiskDecorator.h index 71bb100c576..f1ddfff4952 100644 --- a/src/Disks/DiskDecorator.h +++ b/src/Disks/DiskDecorator.h @@ -42,6 +42,9 @@ public: void setReadOnly(const String & path) override; void createHardLink(const String & src_path, const String & dst_path) override; void truncateFile(const String & path, size_t size) override; + int open(const String & path, mode_t mode) const override; + void close(int fd) const override; + void sync(int fd) const override; const String getType() const override { return delegate->getType(); } protected: diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp index a7486158737..f0738a1130a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp @@ -70,7 +70,7 @@ void MergeTreeDataPartWriterInMemory::calculateAndSerializePrimaryIndex(const Bl } } -void MergeTreeDataPartWriterInMemory::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) +void MergeTreeDataPartWriterInMemory::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool) { /// If part is empty we still need to initialize block by empty columns. if (!part_in_memory->block) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h index 92e4228a90d..6e59cdd08a9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h @@ -18,7 +18,7 @@ public: void write(const Block & block, const IColumn::Permutation * permutation, const Block & primary_key_block, const Block & skip_indexes_block) override; - void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override; + void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) override; void calculateAndSerializePrimaryIndex(const Block & primary_index_block) override; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index f3a72657be5..b05b970da3b 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -251,6 +251,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa new_data_part->minmax_idx = std::move(minmax_idx); new_data_part->is_temp = true; + std::optional sync_guard; if (new_data_part->isStoredOnDisk()) { /// The name could be non-unique in case of stale files from previous runs. @@ -262,12 +263,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa new_data_part->volume->getDisk()->removeRecursive(full_path); } - const auto disk = new_data_part->volume->getDisk(); - disk->createDirectories(full_path); + const auto disk = new_data_part->volume->getDisk(); + disk->createDirectories(full_path); - std::optional sync_guard; - if (data.getSettings()->fsync_part_directory) - sync_guard.emplace(disk, full_path); + if (data.getSettings()->fsync_part_directory) + sync_guard.emplace(disk, full_path); + } /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) From 05a5a13e08f615986815e66b75a231824239f558 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 10 Aug 2020 19:23:58 +0800 Subject: [PATCH 012/625] 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 013/625] 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 014/625] 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 015/625] 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 016/625] 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 017/625] 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 018/625] 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 019/625] 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 020/625] 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 021/625] 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 022/625] 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 023/625] 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 024/625] 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 025/625] 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 25140b9bd5b6421b84ef8586827cc49b9d015e7b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Sep 2020 04:39:36 +0300 Subject: [PATCH 026/625] fsync MergeTree format file --- src/Storages/MergeTree/MergeTreeData.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bbefba70c58..bc668659b6a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -220,6 +220,8 @@ MergeTreeData::MergeTreeData( format_version = min_format_version; auto buf = version_file.second->writeFile(version_file.first); writeIntText(format_version.toUnderType(), *buf); + if (global_context.getSettingsRef().fsync_metadata) + buf->sync(); } else { From 927eb32e882d070ff5ff5446d5b9e0071e2c6f9d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Sep 2020 04:46:40 +0300 Subject: [PATCH 027/625] add test for durability (draft) --- utils/durability-test/create.sql | 1 + utils/durability-test/durability-test.sh | 154 +++++++++++++++++++++++ utils/durability-test/insert.sql | 1 + utils/durability-test/install.sh | 3 + utils/durability-test/sshd_config | 8 ++ utils/durability-test/startup.exp | 23 ++++ 6 files changed, 190 insertions(+) create mode 100644 utils/durability-test/create.sql create mode 100644 utils/durability-test/durability-test.sh create mode 100644 utils/durability-test/insert.sql create mode 100644 utils/durability-test/install.sh create mode 100644 utils/durability-test/sshd_config create mode 100755 utils/durability-test/startup.exp diff --git a/utils/durability-test/create.sql b/utils/durability-test/create.sql new file mode 100644 index 00000000000..1ec394100e2 --- /dev/null +++ b/utils/durability-test/create.sql @@ -0,0 +1 @@ +CREATE TABLE test (a Int, s String) ENGINE = MergeTree ORDER BY a; diff --git a/utils/durability-test/durability-test.sh b/utils/durability-test/durability-test.sh new file mode 100644 index 00000000000..1f47c900f49 --- /dev/null +++ b/utils/durability-test/durability-test.sh @@ -0,0 +1,154 @@ +#!/bin/bash + +URL=http://cloud-images.ubuntu.com/bionic/current +IMAGE=bionic-server-cloudimg-amd64.img +SSH_PORT=11022 +CLICKHOUSE_PORT=9090 +PASSWORD=root + +TABLE_NAME=$1 +CREATE_QUERY=$2 +INSERT_QUERY=$3 + +if [[ -z $TABLE_NAME || -z $CREATE_QUERY || -z $INSERT_QUERY ]]; then + echo "Required 3 arguments: table name, file with create query, file with insert query" + exit 1 +fi + +function run() +{ + sshpass -p $PASSWORD ssh -p $SSH_PORT root@localhost "$1" +} + +function copy() +{ + sshpass -p $PASSWORD scp -r -P $SSH_PORT $1 root@localhost:$2 +} + +function wait_vm_for_start() +{ + echo "Waiting until VM started..." + started=0 + for i in {0..100}; do + run "exit" + if [ $? -eq 0 ]; then + started=1 + break + fi + sleep 1s + done + + if ((started == 0)); then + echo "Can't start or connect to VM." + exit 1 + fi + + echo "Started VM" +} + +function wait_clickhouse_for_start() +{ + echo "Waiting until ClickHouse started..." + started=0 + for i in {0..15}; do + run "clickhouse client --query 'select 1'" + if [ $? -eq 0 ]; then + started=1 + break + fi + sleep 1s + done + + if ((started == 0)); then + echo "Can't start ClickHouse." + fi + + echo "Started ClickHouse" +} + +echo "Downloading image" +curl -O $URL/$IMAGE + +qemu-img resize $IMAGE +10G +virt-customize -a $IMAGE --root-password password:$PASSWORD +virt-copy-in -a $IMAGE sshd_config /etc/ssh + +echo "Starting VM" + +chmod +x ./startup.exp +./startup.exp > qemu.log 2>&1 & + +wait_vm_for_start + +echo "Preparing VM" + +# Resize partition +run "growpart /dev/sda 1 && resize2fs /dev/sda1" + +if [[ -z $CLICKHOUSE_BINARY ]]; then + CLICKHOUSE_BINARY=/usr/bin/clickhouse +fi + +if [[ -z $CLICKHOUSE_CONFIG_DIR ]]; then + CLICKHOUSE_CONFIG_DIR=/etc/clickhouse-server +fi + +echo "Using ClickHouse binary: " $CLICKHOUSE_BINARY +echo "Using ClickHouse config from: " $CLICKHOUSE_CONFIG_DIR + +copy $CLICKHOUSE_BINARY /usr/bin +copy $CLICKHOUSE_CONFIG_DIR /etc +run "mv /etc/$CLICKHOUSE_CONFIG_DIR /etc/clickhouse-server" + +echo "Prepared VM" +echo "Starting ClickHouse" + +run "clickhouse server --config-file=/etc/clickhouse-server/config.xml > clickhouse-server.log 2>&1" & + +wait_clickhouse_for_start + +echo "Started ClickHouse" + +query=`cat $CREATE_QUERY` +echo "Executing query:" $query +run "clickhouse client --query '$query'" + +query=`cat $INSERT_QUERY` +echo "Will run in a loop query: " $query +run "clickhouse benchmark <<< '$query'" & +echo "Running queries" + +pid=`pidof qemu-system-x86_64` +sec=$(( (RANDOM % 3) + 25 )) + +ms=$(( RANDOM % 1000 )) + +echo "Will kill VM in $sec.$ms sec" + +sleep $sec.$ms +kill -9 $pid + +echo "Restarting" + +./startup.exp > qemu.log 2>&1 & +wait_vm_for_start + +run "rm -r *data/system" +run "clickhouse server --config-file=/etc/clickhouse-server/config.xml > clickhouse-server.log 2>&1" & +wait_clickhouse_for_start + +result=`run "grep $TABLE_NAME clickhouse-server.log | grep 'Caught exception while loading metadata'"` +if [[ -n $result ]]; then + echo "FAIL. Can't attach table:" + echo $result + exit 1 +fi + +result=`run "grep $TABLE_NAME clickhouse-server.log | grep 'Considering to remove broken part'"` +if [[ -n $result ]]; then + echo "FAIL. Have broken parts:" + echo $result + exit 1 +fi + +echo OK diff --git a/utils/durability-test/insert.sql b/utils/durability-test/insert.sql new file mode 100644 index 00000000000..8982ad47228 --- /dev/null +++ b/utils/durability-test/insert.sql @@ -0,0 +1 @@ +INSERT INTO test SELECT number, toString(number) FROM numbers(10) diff --git a/utils/durability-test/install.sh b/utils/durability-test/install.sh new file mode 100644 index 00000000000..526cde6743f --- /dev/null +++ b/utils/durability-test/install.sh @@ -0,0 +1,3 @@ +#!/bin/bash + +apt update && apt install qemu-kvm qemu virt-manager virt-viewer libguestfs-tools sshpass expect diff --git a/utils/durability-test/sshd_config b/utils/durability-test/sshd_config new file mode 100644 index 00000000000..6ed06d3d8ad --- /dev/null +++ b/utils/durability-test/sshd_config @@ -0,0 +1,8 @@ +PermitRootLogin yes +PasswordAuthentication yes +ChallengeResponseAuthentication no +UsePAM yes +X11Forwarding yes +PrintMotd no +AcceptEnv LANG LC_* +Subsystem sftp /usr/lib/openssh/sftp-server diff --git a/utils/durability-test/startup.exp b/utils/durability-test/startup.exp new file mode 100755 index 00000000000..540cfc0e4b8 --- /dev/null +++ b/utils/durability-test/startup.exp @@ -0,0 +1,23 @@ +#!/usr/bin/expect -f + +# Wait enough (forever) until a long-time boot +set timeout -1 + +spawn qemu-system-x86_64 \ + -hda bionic-server-cloudimg-amd64.img \ + -cpu qemu64,+ssse3,+sse4.1,+sse4.2,+popcnt -smp 8 \ + -net nic -net user,hostfwd=tcp::11022-:22 \ + -m 4096 -nographic + +expect "login: " +send "root\n" + +expect "Password: " +send "root\n" + +# Without it ssh is not working on guest machine for some reason +expect "# " +send "dhclient && ssh-keygen -A && systemctl restart sshd.service\n" + +# Wait forever +expect "########" From 3cadc9033ae63d7faa851b1707b3c6f9ce1a36aa Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Sep 2020 18:26:49 +0300 Subject: [PATCH 028/625] fsyncs for metadata files of part --- .../MergeTree/IMergeTreeDataPartWriter.h | 2 +- .../MergeTreeDataPartWriterOnDisk.cpp | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 13 +++++++-- .../MergeTree/MergedBlockOutputStream.h | 3 +- utils/durability-test/create_sync.sql | 1 + utils/durability-test/durability-test.sh | 28 ++++++++++--------- utils/durability-test/insert_sync.sql | 1 + 7 files changed, 32 insertions(+), 18 deletions(-) create mode 100644 utils/durability-test/create_sync.sql mode change 100644 => 100755 utils/durability-test/durability-test.sh create mode 100644 utils/durability-test/insert_sync.sql diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 4d3602e732e..4a42a58a65b 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -52,7 +52,7 @@ public: virtual void initPrimaryIndex() {} virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) = 0; - virtual void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & /* checksums */, bool /* sync */) {} + virtual void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & /* checksums */, bool /* sync */) {} virtual void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & /* checksums */, bool /* sync */) {} Columns releaseIndexColumns(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index dbe41144573..8295b881d87 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -332,7 +332,7 @@ void MergeTreeDataPartWriterOnDisk::finishPrimaryIndexSerialization( checksums.files["primary.idx"].file_size = index_stream->count(); checksums.files["primary.idx"].file_hash = index_stream->getHash(); if (sync) - index_stream->sync(); + index_file_stream->sync(); index_stream = nullptr; } } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index fdef5d69688..bdc6bade259 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -111,7 +111,7 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( part_columns = *total_columns_list; if (new_part->isStoredOnDisk()) - finalizePartOnDisk(new_part, part_columns, checksums); + finalizePartOnDisk(new_part, part_columns, checksums, sync); new_part->setColumns(part_columns); new_part->rows_count = rows_count; @@ -126,7 +126,8 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( void MergedBlockOutputStream::finalizePartOnDisk( const MergeTreeData::MutableDataPartPtr & new_part, NamesAndTypesList & part_columns, - MergeTreeData::DataPart::Checksums & checksums) + MergeTreeData::DataPart::Checksums & checksums, + bool sync) { if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || isCompactPart(new_part)) { @@ -143,6 +144,8 @@ void MergedBlockOutputStream::finalizePartOnDisk( count_out_hashing.next(); checksums.files["count.txt"].file_size = count_out_hashing.count(); checksums.files["count.txt"].file_hash = count_out_hashing.getHash(); + if (sync) + count_out->sync(); } if (!new_part->ttl_infos.empty()) @@ -153,6 +156,8 @@ void MergedBlockOutputStream::finalizePartOnDisk( new_part->ttl_infos.write(out_hashing); checksums.files["ttl.txt"].file_size = out_hashing.count(); checksums.files["ttl.txt"].file_hash = out_hashing.getHash(); + if (sync) + out->sync(); } removeEmptyColumnsFromPart(new_part, part_columns, checksums); @@ -161,12 +166,16 @@ void MergedBlockOutputStream::finalizePartOnDisk( /// Write a file with a description of columns. auto out = volume->getDisk()->writeFile(part_path + "columns.txt", 4096); part_columns.writeText(*out); + if (sync) + out->sync(); } { /// Write file with checksums. auto out = volume->getDisk()->writeFile(part_path + "checksums.txt", 4096); checksums.write(*out); + if (sync) + out->sync(); } } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 0b500b93f01..87ff9dd1ded 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -59,7 +59,8 @@ private: void finalizePartOnDisk( const MergeTreeData::MutableDataPartPtr & new_part, NamesAndTypesList & part_columns, - MergeTreeData::DataPart::Checksums & checksums); + MergeTreeData::DataPart::Checksums & checksums, + bool sync); private: NamesAndTypesList columns_list; diff --git a/utils/durability-test/create_sync.sql b/utils/durability-test/create_sync.sql new file mode 100644 index 00000000000..2cc88d2c943 --- /dev/null +++ b/utils/durability-test/create_sync.sql @@ -0,0 +1 @@ +CREATE TABLE test_sync (a Int, s String) ENGINE = MergeTree ORDER BY a SETTINGS fsync_after_insert = 1, min_compressed_bytes_to_fsync_after_merge = 1; diff --git a/utils/durability-test/durability-test.sh b/utils/durability-test/durability-test.sh old mode 100644 new mode 100755 index 1f47c900f49..c7f8936ec95 --- a/utils/durability-test/durability-test.sh +++ b/utils/durability-test/durability-test.sh @@ -17,12 +17,12 @@ fi function run() { - sshpass -p $PASSWORD ssh -p $SSH_PORT root@localhost "$1" + sshpass -p $PASSWORD ssh -p $SSH_PORT root@localhost "$1" 2>/dev/null } function copy() { - sshpass -p $PASSWORD scp -r -P $SSH_PORT $1 root@localhost:$2 + sshpass -p $PASSWORD scp -r -P $SSH_PORT $1 root@localhost:$2 2>/dev/null } function wait_vm_for_start() @@ -50,8 +50,8 @@ function wait_clickhouse_for_start() { echo "Waiting until ClickHouse started..." started=0 - for i in {0..15}; do - run "clickhouse client --query 'select 1'" + for i in {0..30}; do + run "clickhouse client --query 'select 1'" > /dev/null if [ $? -eq 0 ]; then started=1 break @@ -70,7 +70,7 @@ echo "Downloading image" curl -O $URL/$IMAGE qemu-img resize $IMAGE +10G -virt-customize -a $IMAGE --root-password password:$PASSWORD +virt-customize -a $IMAGE --root-password password:$PASSWORD > /dev/null 2>&1 virt-copy-in -a $IMAGE sshd_config /etc/ssh echo "Starting VM" @@ -93,8 +93,8 @@ if [[ -z $CLICKHOUSE_CONFIG_DIR ]]; then CLICKHOUSE_CONFIG_DIR=/etc/clickhouse-server fi -echo "Using ClickHouse binary: " $CLICKHOUSE_BINARY -echo "Using ClickHouse config from: " $CLICKHOUSE_CONFIG_DIR +echo "Using ClickHouse binary:" $CLICKHOUSE_BINARY +echo "Using ClickHouse config from:" $CLICKHOUSE_CONFIG_DIR copy $CLICKHOUSE_BINARY /usr/bin copy $CLICKHOUSE_CONFIG_DIR /etc @@ -104,23 +104,19 @@ echo "Prepared VM" echo "Starting ClickHouse" run "clickhouse server --config-file=/etc/clickhouse-server/config.xml > clickhouse-server.log 2>&1" & - wait_clickhouse_for_start -echo "Started ClickHouse" - query=`cat $CREATE_QUERY` echo "Executing query:" $query run "clickhouse client --query '$query'" query=`cat $INSERT_QUERY` echo "Will run in a loop query: " $query -run "clickhouse benchmark <<< '$query'" & +run "clickhouse benchmark <<< '$query' -c 8" & echo "Running queries" pid=`pidof qemu-system-x86_64` -sec=$(( (RANDOM % 3) + 25 )) - +sec=$(( (RANDOM % 5) + 25 )) ms=$(( RANDOM % 1000 )) echo "Will kill VM in $sec.$ms sec" @@ -130,6 +126,8 @@ kill -9 $pid echo "Restarting" +sleep 5s + ./startup.exp > qemu.log 2>&1 & wait_vm_for_start @@ -137,10 +135,12 @@ run "rm -r *data/system" run "clickhouse server --config-file=/etc/clickhouse-server/config.xml > clickhouse-server.log 2>&1" & wait_clickhouse_for_start +pid=`pidof qemu-system-x86_64` result=`run "grep $TABLE_NAME clickhouse-server.log | grep 'Caught exception while loading metadata'"` if [[ -n $result ]]; then echo "FAIL. Can't attach table:" echo $result + kill -9 $pid exit 1 fi @@ -148,7 +148,9 @@ result=`run "grep $TABLE_NAME clickhouse-server.log | grep 'Considering to remov if [[ -n $result ]]; then echo "FAIL. Have broken parts:" echo $result + kill -9 $pid exit 1 fi +kill -9 $pid echo OK diff --git a/utils/durability-test/insert_sync.sql b/utils/durability-test/insert_sync.sql new file mode 100644 index 00000000000..a1ad2ff4ea5 --- /dev/null +++ b/utils/durability-test/insert_sync.sql @@ -0,0 +1 @@ +INSERT INTO test_sync SELECT number, toString(number) FROM numbers(10) From 26d75f76026303b6f3769ab4ea39ff639ebe836a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 2 Sep 2020 01:25:10 +0300 Subject: [PATCH 029/625] do fsync for WAL --- src/Storages/MergeTree/MergeTreeSettings.h | 2 ++ .../MergeTree/MergeTreeWriteAheadLog.cpp | 32 +++++++++++++++++-- .../MergeTree/MergeTreeWriteAheadLog.h | 10 +++++- 3 files changed, 41 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 1341526c38b..edf03710974 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -43,6 +43,8 @@ struct Settings; M(UInt64, min_compressed_bytes_to_fsync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ M(Bool, fsync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ M(Bool, fsync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ + M(UInt64, write_ahead_log_bytes_to_fsync, 100ULL * 1024 * 1024, "Amount of bytes, accumulated in WAL to do fsync.", 0) \ + M(UInt64, write_ahead_log_interval_ms_to_fsync, 100, "Interval in milliseconds after which fsync for WAL is being done.", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index eda8579c76a..6f220fc7d5d 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -16,17 +17,23 @@ namespace ErrorCodes extern const int CORRUPTED_DATA; } - MergeTreeWriteAheadLog::MergeTreeWriteAheadLog( - const MergeTreeData & storage_, + MergeTreeData & storage_, const DiskPtr & disk_, const String & name_) : storage(storage_) , disk(disk_) , name(name_) , path(storage.getRelativeDataPath() + name_) + , pool(storage.global_context.getSchedulePool()) { init(); + sync_task = pool.createTask("MergeTreeWriteAheadLog::sync", [this] + { + std::lock_guard lock(write_mutex); + out->sync(); + sync_scheduled = false; + }); } void MergeTreeWriteAheadLog::init() @@ -38,6 +45,7 @@ void MergeTreeWriteAheadLog::init() block_out = std::make_unique(*out, 0, Block{}); min_block_number = std::numeric_limits::max(); max_block_number = -1; + bytes_at_last_sync = 0; } void MergeTreeWriteAheadLog::addPart(const Block & block, const String & part_name) @@ -53,6 +61,7 @@ void MergeTreeWriteAheadLog::addPart(const Block & block, const String & part_na writeStringBinary(part_name, *out); block_out->write(block); block_out->flush(); + sync(lock); auto max_wal_bytes = storage.getSettings()->write_ahead_log_max_bytes; if (out->count() > max_wal_bytes) @@ -66,6 +75,7 @@ void MergeTreeWriteAheadLog::dropPart(const String & part_name) writeIntBinary(static_cast(0), *out); writeIntBinary(static_cast(ActionType::DROP_PART), *out); writeStringBinary(part_name, *out); + sync(lock); } void MergeTreeWriteAheadLog::rotate(const std::lock_guard &) @@ -175,6 +185,24 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor return result; } +void MergeTreeWriteAheadLog::sync(const std::lock_guard &) +{ + size_t bytes_to_sync = storage.getSettings()->write_ahead_log_bytes_to_fsync; + time_t time_to_sync = storage.getSettings()->write_ahead_log_interval_ms_to_fsync; + size_t current_bytes = out->count(); + + if (bytes_to_sync && current_bytes - bytes_at_last_sync > bytes_to_sync) + { + sync_task->schedule(); + bytes_at_last_sync = current_bytes; + } + else if (time_to_sync && !sync_scheduled) + { + sync_task->scheduleAfter(time_to_sync); + sync_scheduled = true; + } +} + std::optional MergeTreeWriteAheadLog::tryParseMinMaxBlockNumber(const String & filename) { diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index 2cc3c2b4181..43abf3c04be 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB @@ -31,7 +32,7 @@ public: constexpr static auto WAL_FILE_EXTENSION = ".bin"; constexpr static auto DEFAULT_WAL_FILE_NAME = "wal.bin"; - MergeTreeWriteAheadLog(const MergeTreeData & storage_, const DiskPtr & disk_, + MergeTreeWriteAheadLog(MergeTreeData & storage_, const DiskPtr & disk_, const String & name = DEFAULT_WAL_FILE_NAME); void addPart(const Block & block, const String & part_name); @@ -44,6 +45,7 @@ public: private: void init(); void rotate(const std::lock_guard & lock); + void sync(const std::lock_guard & lock); const MergeTreeData & storage; DiskPtr disk; @@ -56,6 +58,12 @@ private: Int64 min_block_number = std::numeric_limits::max(); Int64 max_block_number = -1; + BackgroundSchedulePool & pool; + BackgroundSchedulePoolTaskHolder sync_task; + + size_t bytes_at_last_sync = 0; + bool sync_scheduled = false; + mutable std::mutex write_mutex; }; From 0db5b4a72ceb27a39b79c4b975f6c119e9057e29 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 2 Sep 2020 02:43:23 +0300 Subject: [PATCH 030/625] 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 031/625] 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 032/625] 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 033/625] 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 034/625] 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 035/625] 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 036/625] 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 037/625] 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 038/625] 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 039/625] 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 040/625] 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 041/625] 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 042/625] 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 043/625] 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 044/625] 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 045/625] 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 046/625] 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 047/625] 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 048/625] 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 049/625] 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 050/625] 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 051/625] 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 052/625] 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 053/625] 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 054/625] 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 055/625] 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 056/625] 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 057/625] 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 058/625] 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 c34eaf5de3380e8b12f0f6e8b578bb13744660bf Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 10:08:38 +0300 Subject: [PATCH 059/625] Update ci_config and llvm --- contrib/llvm | 2 +- tests/ci/ci_config.json | 26 +++++++++++++------------- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/contrib/llvm b/contrib/llvm index 3d6c7e91676..8f24d507c1c 160000 --- a/contrib/llvm +++ b/contrib/llvm @@ -1 +1 @@ -Subproject commit 3d6c7e916760b395908f28a1c885c8334d4fa98b +Subproject commit 8f24d507c1cfeec66d27f48fe74518fd278e2d25 diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index 44e9df49216..adb736a8df3 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -1,7 +1,7 @@ { "build_config": [ { - "compiler": "gcc-9", + "compiler": "gcc-10", "build-type": "", "sanitizer": "", "package-type": "deb", @@ -12,7 +12,7 @@ "with_coverage": false }, { - "compiler": "gcc-9", + "compiler": "gcc-10", "build-type": "", "sanitizer": "", "package-type": "performance", @@ -22,7 +22,7 @@ "with_coverage": false }, { - "compiler": "gcc-9", + "compiler": "gcc-10", "build-type": "", "sanitizer": "", "package-type": "binary", @@ -92,7 +92,7 @@ "with_coverage": false }, { - "compiler": "gcc-9", + "compiler": "gcc-10", "build-type": "", "sanitizer": "", "package-type": "deb", @@ -227,7 +227,7 @@ }, "Functional stateful tests (release)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -239,7 +239,7 @@ }, "Functional stateful tests (release, DatabaseAtomic)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -311,7 +311,7 @@ }, "Functional stateless tests (release)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -323,7 +323,7 @@ }, "Functional stateless tests (unbundled)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -335,7 +335,7 @@ }, "Functional stateless tests (release, polymorphic parts enabled)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -347,7 +347,7 @@ }, "Functional stateless tests (release, DatabaseAtomic)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -443,7 +443,7 @@ }, "Compatibility check": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -467,7 +467,7 @@ }, "Testflows check": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -479,7 +479,7 @@ }, "Unit tests release gcc": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "binary", "build_type": "relwithdebuginfo", "sanitizer": "none", From 4ba8f8960bd4e86a57dafba6a0aa1574b66d97db Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 12:53:24 +0300 Subject: [PATCH 060/625] Increase frame-larger-than --- cmake/warnings.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 2f78dc34079..aec3e46ffa6 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -23,7 +23,7 @@ option (WEVERYTHING "Enables -Weverything option with some exceptions. This is i # Control maximum size of stack frames. It can be important if the code is run in fibers with small stack size. # Only in release build because debug has too large stack frames. if ((NOT CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") AND (NOT SANITIZE)) - add_warning(frame-larger-than=16384) + add_warning(frame-larger-than=32768) endif () if (COMPILER_CLANG) From f528cd9f97b4f7c54a6c22406f09983d055ce642 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 13:01:12 +0300 Subject: [PATCH 061/625] Forward compiler version to unbundled build --- docker/packager/packager | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/packager b/docker/packager/packager index 5874bedd17a..909f20acd6d 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -93,7 +93,7 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ cxx = cc.replace('gcc', 'g++').replace('clang', 'clang++') - if image_type == "deb": + if image_type == "deb" or image_type == "unbundled": result.append("DEB_CC={}".format(cc)) result.append("DEB_CXX={}".format(cxx)) elif image_type == "binary": From ca6b634eb0466361da6f3526a6611ab0ccd8bfc1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 13:51:01 +0300 Subject: [PATCH 062/625] Install gcc-10 from proposed repo --- docker/packager/binary/Dockerfile | 13 +++++++++++-- docker/packager/deb/Dockerfile | 12 ++++++++++-- 2 files changed, 21 insertions(+), 4 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 45c35c2e0f3..b911b59a41d 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -32,8 +32,6 @@ RUN apt-get update \ curl \ gcc-9 \ g++-9 \ - gcc-10 \ - g++-10 \ llvm-${LLVM_VERSION} \ clang-${LLVM_VERSION} \ lld-${LLVM_VERSION} \ @@ -93,5 +91,16 @@ RUN wget -nv "https://developer.arm.com/-/media/Files/downloads/gnu-a/8.3-2019.0 # Download toolchain for FreeBSD 11.3 RUN wget -nv https://clickhouse-datasets.s3.yandex.net/toolchains/toolchains/freebsd-11.3-toolchain.tar.xz +# NOTE: For some reason we have outdated version of gcc-10 in ubuntu 20.04 stable. +# Current workaround is to use latest version proposed repo. Remove as soon as +# gcc-10.2 appear in stable repo. +RUN echo 'deb http://archive.ubuntu.com/ubuntu/ focal-proposed restricted main multiverse universe' > /etc/apt/sources.list.d/proposed-repositories.list + +RUN apt-get update \ + && apt-get install gcc-10 g++10 --yes + +RUN rm /etc/apt/sources.list.d/proposed-repositories.list + + COPY build.sh / CMD ["/bin/bash", "/build.sh"] diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index 87f4582f8e2..30334504c55 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -42,8 +42,6 @@ RUN export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ # Libraries from OS are only needed to test the "unbundled" build (this is not used in production). RUN apt-get update \ && apt-get install \ - gcc-10 \ - g++-10 \ gcc-9 \ g++-9 \ clang-11 \ @@ -75,6 +73,16 @@ RUN apt-get update \ pigz \ --yes --no-install-recommends +# NOTE: For some reason we have outdated version of gcc-10 in ubuntu 20.04 stable. +# Current workaround is to use latest version proposed repo. Remove as soon as +# gcc-10.2 appear in stable repo. +RUN echo 'deb http://archive.ubuntu.com/ubuntu/ focal-proposed restricted main multiverse universe' > /etc/apt/sources.list.d/proposed-repositories.list + +RUN apt-get update \ + && apt-get install gcc-10 g++10 --yes --no-install-recommends + +RUN rm /etc/apt/sources.list.d/proposed-repositories.list + # This symlink required by gcc to find lld compiler RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld From c535d752438c9616dab8fac79bf8594acb44665a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 14:47:34 +0300 Subject: [PATCH 063/625] Add update --- docker/packager/binary/Dockerfile | 2 +- docker/packager/deb/Dockerfile | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index b911b59a41d..893e9191b1e 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -99,7 +99,7 @@ RUN echo 'deb http://archive.ubuntu.com/ubuntu/ focal-proposed restricted main m RUN apt-get update \ && apt-get install gcc-10 g++10 --yes -RUN rm /etc/apt/sources.list.d/proposed-repositories.list +RUN rm /etc/apt/sources.list.d/proposed-repositories.list && apt-get update COPY build.sh / diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index 30334504c55..4b7c2ae53a4 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -81,7 +81,7 @@ RUN echo 'deb http://archive.ubuntu.com/ubuntu/ focal-proposed restricted main m RUN apt-get update \ && apt-get install gcc-10 g++10 --yes --no-install-recommends -RUN rm /etc/apt/sources.list.d/proposed-repositories.list +RUN rm /etc/apt/sources.list.d/proposed-repositories.list && apt-get update # This symlink required by gcc to find lld compiler RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld From c03a9487ca542b93fa51db1a671f1f365da0c081 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 9 Sep 2020 14:55:20 +0300 Subject: [PATCH 064/625] 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 065/625] =?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 066/625] 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 067/625] 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 068/625] 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 956138635de536560d0843025720d7ce7b947cf3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 15:59:26 +0300 Subject: [PATCH 069/625] Fix compiler name --- docker/packager/binary/Dockerfile | 2 +- docker/packager/deb/Dockerfile | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 893e9191b1e..03bb3b5aefa 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -97,7 +97,7 @@ RUN wget -nv https://clickhouse-datasets.s3.yandex.net/toolchains/toolchains/fre RUN echo 'deb http://archive.ubuntu.com/ubuntu/ focal-proposed restricted main multiverse universe' > /etc/apt/sources.list.d/proposed-repositories.list RUN apt-get update \ - && apt-get install gcc-10 g++10 --yes + && apt-get install gcc-10 g++-10 --yes RUN rm /etc/apt/sources.list.d/proposed-repositories.list && apt-get update diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index 4b7c2ae53a4..a3c87f13fe4 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -79,7 +79,7 @@ RUN apt-get update \ RUN echo 'deb http://archive.ubuntu.com/ubuntu/ focal-proposed restricted main multiverse universe' > /etc/apt/sources.list.d/proposed-repositories.list RUN apt-get update \ - && apt-get install gcc-10 g++10 --yes --no-install-recommends + && apt-get install gcc-10 g++-10 --yes --no-install-recommends RUN rm /etc/apt/sources.list.d/proposed-repositories.list && apt-get update From b68782d285e5ea76f7318b55bf41cf337dfa71fc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 9 Sep 2020 16:32:50 +0300 Subject: [PATCH 070/625] enable more tests with Atomic database --- docker/test/stress/stress | 2 +- programs/client/Client.cpp | 28 +++++++- src/Interpreters/DatabaseCatalog.cpp | 5 +- src/Interpreters/InterpreterCreateQuery.cpp | 1 + .../MergeTree/MergeTreeWriteAheadLog.cpp | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 16 +++-- src/Storages/System/StorageSystemTables.cpp | 6 ++ .../queries/0_stateless/00116_storage_set.sql | 2 +- .../00180_attach_materialized_view.sql | 2 +- ...per_deduplication_and_unexpected_parts.sql | 2 +- .../00281_compile_sizeof_packed.re | 0 .../0_stateless/00311_array_primary_key.sql | 2 +- .../00423_storage_log_single_thread.sql | 6 +- .../00816_long_concurrent_alter_column.sh | 27 +++++--- .../01190_full_attach_syntax.reference | 13 ++++ .../0_stateless/01190_full_attach_syntax.sql | 66 +++++++++++++++++++ .../01305_replica_create_drop_zookeeper.sh | 20 ++++-- .../00065_loyalty_with_storage_join.sql | 2 +- tests/queries/skip_list.json | 33 +--------- 19 files changed, 172 insertions(+), 62 deletions(-) delete mode 100644 tests/queries/0_stateless/00281_compile_sizeof_packed.re create mode 100644 tests/queries/0_stateless/01190_full_attach_syntax.reference create mode 100644 tests/queries/0_stateless/01190_full_attach_syntax.sql diff --git a/docker/test/stress/stress b/docker/test/stress/stress index e8675da1546..60db5ec465c 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -28,7 +28,7 @@ def get_options(i): options = "" if 0 < i: options += " --order=random" - if i == 1: + if i % 2 == 1: options += " --atomic-db-engine" return options diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index c9701950dc5..83e4062b1f3 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -919,7 +919,33 @@ private: while (begin < end) { const char * pos = begin; - ASTPtr orig_ast = parseQuery(pos, end, true); + + ASTPtr orig_ast; + try + { + orig_ast = parseQuery(pos, end, true); + } + catch (Exception & e) + { + if (!test_mode) + throw; + + /// Try find test hint for syntax error + const char * end_of_line = find_first_symbols<'\n'>(begin, end); + TestHint hint(true, String(begin, end_of_line - begin)); + if (hint.serverError()) /// Syntax errors are considered as client errors + throw; + if (hint.clientError() != e.code()) + { + if (hint.clientError()) + e.addMessage("\nExpected clinet error: " + std::to_string(hint.clientError())); + throw; + } + + /// It's expected syntax error, skip the line + begin = end_of_line; + continue; + } if (!orig_ast) { diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 6153f6b52fb..049341918b9 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -657,7 +657,10 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr /// Table was removed from database. Enqueue removal of its data from disk. time_t drop_time; if (table) + { drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + table->is_dropped = true; + } else { /// Try load table from metadata to drop it correctly (e.g. remove metadata from zk or remove data from all volumes) @@ -674,6 +677,7 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr try { table = createTableFromAST(*create, table_id.getDatabaseName(), data_path, *global_context, false).second; + table->is_dropped = true; } catch (...) { @@ -763,7 +767,6 @@ void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table) const if (table.table) { table.table->drop(); - table.table->is_dropped = true; } /// Even if table is not loaded, try remove its data from disk. diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 06973ab029b..d7230940bb2 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -673,6 +673,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) create.attach_short_syntax = true; create.if_not_exists = if_not_exists; } + /// TODO maybe assert table structure if create.attach_short_syntax is false? if (!create.temporary && create.database.empty()) create.database = current_database; diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 53ef72f3208..3fa3a7e3e40 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -66,6 +66,7 @@ void MergeTreeWriteAheadLog::dropPart(const String & part_name) writeIntBinary(static_cast(0), *out); writeIntBinary(static_cast(ActionType::DROP_PART), *out); writeStringBinary(part_name, *out); + out->next(); } void MergeTreeWriteAheadLog::rotate(const std::lock_guard &) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6058632d220..6458fe127da 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4260,9 +4260,13 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( * To do this, check its node `log_pointer` - the maximum number of the element taken from `log` + 1. */ - const auto & check_replica_become_inactive = [this, &replica]() + bool waiting_itself = replica == replica_name; + + const auto & stop_waiting = [&]() { - return !getZooKeeper()->exists(zookeeper_path + "/replicas/" + replica + "/is_active"); + bool stop_waiting_itself = waiting_itself && is_dropped; + bool stop_waiting_non_active = !wait_for_non_active && !getZooKeeper()->exists(zookeeper_path + "/replicas/" + replica + "/is_active"); + return stop_waiting_itself || stop_waiting_non_active; }; constexpr auto event_wait_timeout_ms = 1000; @@ -4277,7 +4281,7 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( LOG_DEBUG(log, "Waiting for {} to pull {} to queue", replica, log_node_name); /// Let's wait until entry gets into the replica queue. - while (wait_for_non_active || !check_replica_become_inactive()) + while (!stop_waiting()) { zkutil::EventPtr event = std::make_shared(); @@ -4325,7 +4329,7 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( LOG_DEBUG(log, "Waiting for {} to pull {} to queue", replica, log_node_name); /// Let's wait until the entry gets into the replica queue. - while (wait_for_non_active || !check_replica_become_inactive()) + while (!stop_waiting()) { zkutil::EventPtr event = std::make_shared(); @@ -4378,10 +4382,8 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( /// Third - wait until the entry disappears from the replica queue or replica become inactive. String path_to_wait_on = zookeeper_path + "/replicas/" + replica + "/queue/" + queue_entry_to_wait_for; - if (wait_for_non_active) - return getZooKeeper()->waitForDisappear(path_to_wait_on); - return getZooKeeper()->waitForDisappear(path_to_wait_on, check_replica_become_inactive); + return getZooKeeper()->waitForDisappear(path_to_wait_on, stop_waiting); } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 5b7dad836e9..0ad961ad7d8 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -344,6 +344,12 @@ protected: { ASTPtr ast = database->tryGetCreateTableQuery(table_name, context); + if (ast && !context.getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil) + { + auto & create = ast->as(); + create.uuid = UUIDHelpers::Nil; + } + if (columns_mask[src_index++]) res_columns[res_index++]->insert(ast ? queryToString(ast) : ""); diff --git a/tests/queries/0_stateless/00116_storage_set.sql b/tests/queries/0_stateless/00116_storage_set.sql index aa93a0620d0..0eeed7e859a 100644 --- a/tests/queries/0_stateless/00116_storage_set.sql +++ b/tests/queries/0_stateless/00116_storage_set.sql @@ -19,7 +19,7 @@ INSERT INTO set2 VALUES ('abc'), ('World'); SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2; DETACH TABLE set2; -ATTACH TABLE set2 (x String) ENGINE = Set; +ATTACH TABLE set2; SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2; diff --git a/tests/queries/0_stateless/00180_attach_materialized_view.sql b/tests/queries/0_stateless/00180_attach_materialized_view.sql index 089e4926bcf..d674c0bd277 100644 --- a/tests/queries/0_stateless/00180_attach_materialized_view.sql +++ b/tests/queries/0_stateless/00180_attach_materialized_view.sql @@ -6,7 +6,7 @@ CREATE TABLE t_00180 (x UInt8) ENGINE = Null; CREATE MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180; DETACH TABLE mv_00180; -ATTACH MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180; +ATTACH TABLE mv_00180; DROP TABLE t_00180; DROP TABLE mv_00180; diff --git a/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql b/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql index 623218af167..c14ce53d4a3 100644 --- a/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql +++ b/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql @@ -21,7 +21,7 @@ INSERT INTO deduplication (x) VALUES (1); SELECT * FROM deduplication; DETACH TABLE deduplication; -ATTACH TABLE deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00226/deduplication', 'r1', d, x, 1); +ATTACH TABLE deduplication; SELECT * FROM deduplication; diff --git a/tests/queries/0_stateless/00281_compile_sizeof_packed.re b/tests/queries/0_stateless/00281_compile_sizeof_packed.re deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/00311_array_primary_key.sql b/tests/queries/0_stateless/00311_array_primary_key.sql index 0ea368609da..0e066c64f89 100644 --- a/tests/queries/0_stateless/00311_array_primary_key.sql +++ b/tests/queries/0_stateless/00311_array_primary_key.sql @@ -11,7 +11,7 @@ INSERT INTO array_pk VALUES ([5, 6], 'ghi', 6); SELECT * FROM array_pk ORDER BY n; DETACH TABLE array_pk; -ATTACH TABLE array_pk (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1); +ATTACH TABLE array_pk; SELECT * FROM array_pk ORDER BY n; diff --git a/tests/queries/0_stateless/00423_storage_log_single_thread.sql b/tests/queries/0_stateless/00423_storage_log_single_thread.sql index 7d5e14c9ee5..8eff9323564 100644 --- a/tests/queries/0_stateless/00423_storage_log_single_thread.sql +++ b/tests/queries/0_stateless/00423_storage_log_single_thread.sql @@ -5,7 +5,7 @@ SELECT * FROM log LIMIT 1; SELECT * FROM log; DETACH TABLE log; -ATTACH TABLE log (s String) ENGINE = Log; +ATTACH TABLE log; SELECT * FROM log; SELECT * FROM log LIMIT 1; @@ -15,13 +15,13 @@ INSERT INTO log VALUES ('Hello'), ('World'); SELECT * FROM log LIMIT 1; DETACH TABLE log; -ATTACH TABLE log (s String) ENGINE = Log; +ATTACH TABLE log; SELECT * FROM log LIMIT 1; SELECT * FROM log; DETACH TABLE log; -ATTACH TABLE log (s String) ENGINE = Log; +ATTACH TABLE log; SELECT * FROM log; SELECT * FROM log LIMIT 1; diff --git a/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh b/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh index 93421e003f6..8fdd6654bae 100755 --- a/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh +++ b/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh @@ -11,34 +11,34 @@ echo "CREATE TABLE concurrent_alter_column (ts DATETIME) ENGINE = MergeTree PART function thread1() { while true; do - for i in {1..500}; do echo "ALTER TABLE concurrent_alter_column ADD COLUMN c$i DOUBLE;"; done | ${CLICKHOUSE_CLIENT} -n --query_id=alter1 + for i in {1..500}; do echo "ALTER TABLE concurrent_alter_column ADD COLUMN c$i DOUBLE;"; done | ${CLICKHOUSE_CLIENT} -n --query_id=alter_00816_1 done } function thread2() { while true; do - echo "ALTER TABLE concurrent_alter_column ADD COLUMN d DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter2; + echo "ALTER TABLE concurrent_alter_column ADD COLUMN d DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_2; sleep "$(echo 0.0$RANDOM)"; - echo "ALTER TABLE concurrent_alter_column DROP COLUMN d" | ${CLICKHOUSE_CLIENT} --query_id=alter2; + echo "ALTER TABLE concurrent_alter_column DROP COLUMN d" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_2; done } function thread3() { while true; do - echo "ALTER TABLE concurrent_alter_column ADD COLUMN e DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter3; + echo "ALTER TABLE concurrent_alter_column ADD COLUMN e DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_3; sleep "$(echo 0.0$RANDOM)"; - echo "ALTER TABLE concurrent_alter_column DROP COLUMN e" | ${CLICKHOUSE_CLIENT} --query_id=alter3; + echo "ALTER TABLE concurrent_alter_column DROP COLUMN e" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_3; done } function thread4() { while true; do - echo "ALTER TABLE concurrent_alter_column ADD COLUMN f DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter4; + echo "ALTER TABLE concurrent_alter_column ADD COLUMN f DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_4; sleep "$(echo 0.0$RANDOM)"; - echo "ALTER TABLE concurrent_alter_column DROP COLUMN f" | ${CLICKHOUSE_CLIENT} --query_id=alter4; + echo "ALTER TABLE concurrent_alter_column DROP COLUMN f" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_4; done } @@ -57,9 +57,18 @@ timeout $TIMEOUT bash -c thread4 2> /dev/null & wait -echo "DROP TABLE concurrent_alter_column" | ${CLICKHOUSE_CLIENT} +echo "DROP TABLE concurrent_alter_column NO DELAY" | ${CLICKHOUSE_CLIENT} # NO DELAY has effect only for Atomic database + +db_engine=`$CLICKHOUSE_CLIENT -q "SELECT engine FROM system.databases WHERE name=currentDatabase()"` +if [[ $db_engine == "Atomic" ]]; then + # DROP is non-blocking, so wait for alters + while true; do + $CLICKHOUSE_CLIENT -q "SELECT c = 0 FROM (SELECT count() as c FROM system.processes WHERE query_id LIKE 'alter_00816_%')" | grep 1 > /dev/null && break; + sleep 1; + done +fi # Check for deadlocks -echo "SELECT * FROM system.processes WHERE query_id LIKE 'alter%'" | ${CLICKHOUSE_CLIENT} +echo "SELECT * FROM system.processes WHERE query_id LIKE 'alter_00816_%'" | ${CLICKHOUSE_CLIENT} echo 'did not crash' diff --git a/tests/queries/0_stateless/01190_full_attach_syntax.reference b/tests/queries/0_stateless/01190_full_attach_syntax.reference new file mode 100644 index 00000000000..619861849c8 --- /dev/null +++ b/tests/queries/0_stateless/01190_full_attach_syntax.reference @@ -0,0 +1,13 @@ +CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) +CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) +CREATE TABLE default.log\n(\n `s` String\n)\nENGINE = Log +CREATE TABLE default.log\n(\n `s` String\n)\nENGINE = Log() +test +CREATE TABLE default.mt\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date MATERIALIZED \'2000-01-01\'\n)\nENGINE = MergeTree(d, (key, s, n), 1) +[1,2] Hello 2 +CREATE TABLE default.mt\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date\n)\nENGINE = MergeTree(d, (key, s, n), 1) +CREATE MATERIALIZED VIEW default.mv\n(\n `s` String\n)\nENGINE = Null AS\nSELECT *\nFROM default.log +CREATE MATERIALIZED VIEW default.mv\n(\n `s` String\n)\nENGINE = Null AS\nSELECT *\nFROM default.log +CREATE MATERIALIZED VIEW default.mv\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date\n)\nENGINE = Null AS\nSELECT *\nFROM default.mt +CREATE LIVE VIEW default.lv\n(\n `1` UInt8\n) AS\nSELECT 1 +CREATE LIVE VIEW default.lv\n(\n `1` UInt8\n) AS\nSELECT 1 diff --git a/tests/queries/0_stateless/01190_full_attach_syntax.sql b/tests/queries/0_stateless/01190_full_attach_syntax.sql new file mode 100644 index 00000000000..3a91eccc8cd --- /dev/null +++ b/tests/queries/0_stateless/01190_full_attach_syntax.sql @@ -0,0 +1,66 @@ +DROP DATABASE IF EXISTS test_01190; +CREATE DATABASE test_01190; + +CREATE TABLE test_01190.table_for_dict (key UInt64, col UInt8) ENGINE = Memory; + +CREATE DICTIONARY test_01190.dict (key UInt64 DEFAULT 0, col UInt8 DEFAULT 1) PRIMARY KEY key SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'test_01190')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); + +SHOW CREATE DICTIONARY test_01190.dict; + +DETACH DICTIONARY test_01190.dict; +ATTACH TABLE test_01190.dict; -- { serverError 80 } +-- Full ATTACH syntax is not allowed for dictionaries +ATTACH DICTIONARY test_01190.dict (key UInt64 DEFAULT 0, col UInt8 DEFAULT 42) PRIMARY KEY key SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'test_01190')) LIFETIME(MIN 1 MAX 100) LAYOUT(FLAT()); -- { clientError 62 } +ATTACH DICTIONARY test_01190.dict; +SHOW CREATE DICTIONARY test_01190.dict; + +DROP DATABASE test_01190; + + +DROP TABLE IF EXISTS log; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS mv; +DROP TABLE IF EXISTS lv; + +CREATE TABLE log ENGINE = Log AS SELECT 'test' AS s; +SHOW CREATE log; +DETACH TABLE log; +ATTACH DICTIONARY log; -- { serverError 487 } +ATTACH TABLE log (s String) ENGINE = Log(); +SHOW CREATE log; +SELECT * FROM log; + +DROP TABLE IF EXISTS mt; +CREATE TABLE mt (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1); +INSERT INTO mt VALUES ([1, 2], 'Hello', 2); +DETACH TABLE mt; +ATTACH TABLE mt (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree ORDER BY (key, s, n) PARTITION BY toYYYYMM(d); -- { serverError 342 } +ATTACH TABLE mt (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1); +SHOW CREATE mt; +SELECT * FROM mt; +DETACH TABLE mt; +ATTACH TABLE mt (key Array(UInt8), s String, n UInt64, d Date) ENGINE = MergeTree(d, (key, s, n), 1); -- It works (with Ordinary database), but probably it shouldn't +SHOW CREATE mt; + +CREATE MATERIALIZED VIEW mv ENGINE = Null AS SELECT * FROM log; +SHOW CREATE mv; +DETACH VIEW mv; +ATTACH MATERIALIZED VIEW mv ENGINE = Null AS SELECT * FROM log; +SHOW CREATE mv; +DETACH VIEW mv; +ATTACH MATERIALIZED VIEW mv ENGINE = Null AS SELECT * FROM mt; -- It works (with Ordinary database), but probably it shouldn't +SHOW CREATE mv; + +SET allow_experimental_live_view = 1; +CREATE LIVE VIEW lv AS SELECT 1; +SHOW CREATE lv; +DETACH VIEW lv; +ATTACH LIVE VIEW lv AS SELECT 1; +SHOW CREATE lv; + +DROP TABLE log; +DROP TABLE mt; +DROP TABLE mv; +DROP TABLE lv; + + diff --git a/tests/queries/0_stateless/01305_replica_create_drop_zookeeper.sh b/tests/queries/0_stateless/01305_replica_create_drop_zookeeper.sh index 0a47c6df46c..1313830d589 100755 --- a/tests/queries/0_stateless/01305_replica_create_drop_zookeeper.sh +++ b/tests/queries/0_stateless/01305_replica_create_drop_zookeeper.sh @@ -7,11 +7,21 @@ set -e function thread() { - while true; do - $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS test_table_$1; - CREATE TABLE test_table_$1 (a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01305/alter_table', 'r_$1') ORDER BY tuple();" 2>&1 | - grep -vP '(^$)|(^Received exception from server)|(^\d+\. )|because the last replica of the table was dropped right now|is already started to be removing by another replica right now|is already finished removing by another replica right now|Removing leftovers from table|Another replica was suddenly created|was successfully removed from ZooKeeper|was created by another server at the same moment|was suddenly removed|some other replicas were created at the same time' - done + db_engine=`$CLICKHOUSE_CLIENT -q "SELECT engine FROM system.databases WHERE name=currentDatabase()"` + if [[ $db_engine == "Atomic" ]]; then + # Ignore "Replica already exists" exception + while true; do + $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS test_table_$1 NO DELAY; + CREATE TABLE test_table_$1 (a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01305/alter_table', 'r_$1') ORDER BY tuple();" 2>&1 | + grep -vP '(^$)|(^Received exception from server)|(^\d+\. )|because the last replica of the table was dropped right now|is already started to be removing by another replica right now|is already finished removing by another replica right now|Removing leftovers from table|Another replica was suddenly created|was successfully removed from ZooKeeper|was created by another server at the same moment|was suddenly removed|some other replicas were created at the same time|already exists' + done + else + while true; do + $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS test_table_$1; + CREATE TABLE test_table_$1 (a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01305/alter_table', 'r_$1') ORDER BY tuple();" 2>&1 | + grep -vP '(^$)|(^Received exception from server)|(^\d+\. )|because the last replica of the table was dropped right now|is already started to be removing by another replica right now|is already finished removing by another replica right now|Removing leftovers from table|Another replica was suddenly created|was successfully removed from ZooKeeper|was created by another server at the same moment|was suddenly removed|some other replicas were created at the same time' + done + fi } diff --git a/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql b/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql index 15a2a75cf58..515a2410583 100644 --- a/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql +++ b/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql @@ -22,7 +22,7 @@ GROUP BY loyalty ORDER BY loyalty ASC; DETACH TABLE join; -ATTACH TABLE join (UserID UInt64, loyalty Int8) ENGINE = Join(SEMI, LEFT, UserID); +ATTACH TABLE join; SELECT loyalty, diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index adfc5f0e582..efd622402b2 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -3,10 +3,8 @@ */ { "thread-sanitizer": [ - "00281", "00877", "00985", - "avx2", "query_profiler", "memory_profiler", /// 01083 and 00505 and 00505 are critical and temproray disabled @@ -21,9 +19,7 @@ "01193_metadata_loading" ], "address-sanitizer": [ - "00281", "00877", - "avx2", "query_profiler", "memory_profiler", "odbc_roundtrip", @@ -31,9 +27,7 @@ "01193_metadata_loading" ], "ub-sanitizer": [ - "00281", "capnproto", - "avx2", "query_profiler", "memory_profiler", "01103_check_cpu_instructions_at_startup", @@ -41,9 +35,7 @@ "01193_metadata_loading" ], "memory-sanitizer": [ - "00281", "capnproto", - "avx2", "query_profiler", "memory_profiler", "01103_check_cpu_instructions_at_startup", @@ -53,8 +45,6 @@ "01193_metadata_loading" ], "debug-build": [ - "00281", - "avx2", "query_profiler", "memory_profiler", "00899_long_attach", @@ -70,12 +60,10 @@ ], "unbundled-build": [ "00429", - "00428", "00877", "pocopatch", "parquet", "xxhash", - "avx2", "_h3", "query_profiler", "memory_profiler", @@ -98,33 +86,19 @@ "01455_time_zones" ], "release-build": [ - "avx2" ], "database-atomic": [ - "00065_loyalty_with_storage_join", - "avx", /// Inner tables of materialized views have different names "00738_lock_for_inner_table", - "00699_materialized_view_mutations", "00609_mv_index_in_in", "00510_materizlized_view_and_deduplication_zookeeper", - /// Create queries contain UUID + /// Different database engine "00604_show_create_database", - "00080_show_tables_and_system_tables", - "01272_suspicious_codecs", /// UUID must be specified in ATTACH TABLE - "01249_bad_arguments_for_bloom_filter", - "00423_storage_log_single_thread", - "00311_array_primary_key", - "00226_zookeeper_deduplication_and_unexpected_parts", - "00180_attach_materialized_view", - "00116_storage_set", + "01190_full_attach_syntax", /// Assumes blocking DROP - "00816_long_concurrent_alter_column", - "00992_system_parts_race_condition_zookeeper", /// FIXME "01320_create_sync_race_condition", - "01305_replica_create_drop_zookeeper", - "01130_in_memory_parts_partitons", + /// Internal distionary name is different "01225_show_create_table_from_dictionary", "01224_no_superfluous_dict_reload" ], @@ -132,7 +106,6 @@ /// 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", From 751e4109152671ece957a0c36af43f0e909f41d7 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Wed, 9 Sep 2020 17:11:08 +0300 Subject: [PATCH 071/625] 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 50dee3f4493d7ffb2c75d195cc39862f8f8d8a86 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 17:43:17 +0300 Subject: [PATCH 072/625] Remove false-positive warning --- src/Storages/MergeTree/MergeTreePartition.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 4a846f63b7c..8ef3e458871 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -29,6 +29,9 @@ String MergeTreePartition::getID(const MergeTreeData & storage) const return getID(storage.getInMemoryMetadataPtr()->getPartitionKey().sample_block); } +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wstringop-overflow" + /// NOTE: This ID is used to create part names which are then persisted in ZK and as directory names on the file system. /// So if you want to change this method, be sure to guarantee compatibility with existing table data. String MergeTreePartition::getID(const Block & partition_key_sample) const @@ -87,6 +90,8 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const return result; } +#pragma GCC diagnostic pop + void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); 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 073/625] 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 074/625] 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 7f4106687cb14491246f218654ed8a0a3b751b29 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 9 Sep 2020 19:23:31 +0300 Subject: [PATCH 075/625] fix --- tests/queries/0_stateless/01114_database_atomic.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01114_database_atomic.reference b/tests/queries/0_stateless/01114_database_atomic.reference index 7980819f9af..a79784230a6 100644 --- a/tests/queries/0_stateless/01114_database_atomic.reference +++ b/tests/queries/0_stateless/01114_database_atomic.reference @@ -7,7 +7,7 @@ test_01114_3 Ordinary test_01114_3 test_01114_3 1 20 100 CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -mt 00001114-0000-4000-8000-000000000002 CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\' (`n` UInt64) ENGINE = MergeTree() PARTITION BY n % 5 ORDER BY tuple() SETTINGS index_granularity = 8192 +mt 00001114-0000-4000-8000-000000000002 CREATE TABLE test_01114_2.mt (`n` UInt64) ENGINE = MergeTree() PARTITION BY n % 5 ORDER BY tuple() SETTINGS index_granularity = 8192 20 CREATE TABLE test_01114_1.mt UUID \'00001114-0000-4000-8000-000000000001\'\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 From 62428845a0fdcaaa19ecc5fd33f3ecd849104cf5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 19:47:06 +0300 Subject: [PATCH 076/625] Bug in mutation --- src/Columns/ColumnVector.h | 7 ++++--- .../0_stateless/01475_mutation_with_if.reference | 1 + .../0_stateless/01475_mutation_with_if.sql | 16 ++++++++++++++++ 3 files changed, 21 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01475_mutation_with_if.reference create mode 100644 tests/queries/0_stateless/01475_mutation_with_if.sql diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 1090de556a0..55ab67d6214 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -130,7 +131,7 @@ public: void insertFrom(const IColumn & src, size_t n) override { - data.push_back(static_cast(src).getData()[n]); + data.push_back(assert_cast(src).getData()[n]); } void insertData(const char * pos, size_t) override @@ -205,14 +206,14 @@ public: /// This method implemented in header because it could be possibly devirtualized. int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override { - return CompareHelper::compare(data[n], static_cast(rhs_).data[m], nan_direction_hint); + return CompareHelper::compare(data[n], assert_cast(rhs_).data[m], nan_direction_hint); } void compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override { - return this->template doCompareColumn(static_cast(rhs), rhs_row_num, row_indexes, + return this->template doCompareColumn(assert_cast(rhs), rhs_row_num, row_indexes, compare_results, direction, nan_direction_hint); } diff --git a/tests/queries/0_stateless/01475_mutation_with_if.reference b/tests/queries/0_stateless/01475_mutation_with_if.reference new file mode 100644 index 00000000000..2874a18147f --- /dev/null +++ b/tests/queries/0_stateless/01475_mutation_with_if.reference @@ -0,0 +1 @@ +1 150 diff --git a/tests/queries/0_stateless/01475_mutation_with_if.sql b/tests/queries/0_stateless/01475_mutation_with_if.sql new file mode 100644 index 00000000000..6f0ef8924be --- /dev/null +++ b/tests/queries/0_stateless/01475_mutation_with_if.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS mutation_table; +CREATE TABLE mutation_table ( + id int, + price Nullable(Int32) +) +ENGINE = MergeTree() +PARTITION BY id +ORDER BY id; + +INSERT INTO mutation_table (id, price) VALUES (1, 100); + +ALTER TABLE mutation_table UPDATE price = 150 WHERE id = 1 SETTINGS mutations_sync = 2; + +SELECT * FROM mutation_table; + +DROP TABLE IF EXISTS mutation_table; From e91d120e1a9c8326c895fcb45c9e43f5cdedfdd1 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Wed, 9 Sep 2020 20:11:23 +0300 Subject: [PATCH 077/625] Bump CI. From b8a2c1d2a29517c2bd0e8f791ce31c474f30f7d5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 22:45:37 +0300 Subject: [PATCH 078/625] Push pragma only for new gcc --- src/Storages/MergeTree/MergeTreePartition.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 8ef3e458871..2802b842f54 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -29,8 +29,10 @@ String MergeTreePartition::getID(const MergeTreeData & storage) const return getID(storage.getInMemoryMetadataPtr()->getPartitionKey().sample_block); } -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wstringop-overflow" +#if defined (__GNUC__) && __GNUC__ >= 10 + #pragma GCC diagnostic push + #pragma GCC diagnostic ignored "-Wstringop-overflow" +#endif /// NOTE: This ID is used to create part names which are then persisted in ZK and as directory names on the file system. /// So if you want to change this method, be sure to guarantee compatibility with existing table data. @@ -90,7 +92,9 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const return result; } -#pragma GCC diagnostic pop +#if defined (__GNUC__) && __GNUC__ >= 10 + #pragma GCC diagnostic pop +#endif void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const { From 27258c8e70213cf57e1bbf36176cda961d56e12f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 9 Sep 2020 23:47:42 +0300 Subject: [PATCH 079/625] utils/list-licenses/list-licenses.sh: ignore more files - *.rtf They can have NULL byte, and StorageSystemLicenses.sh will warn: ./StorageSystemLicenses.sh: line 11: warning: command substitution: ignored null byte in input Found with: find contrib/ -type f -and '(' -iname 'LICENSE*' -or -iname 'COPYING*' -or -iname 'COPYRIGHT*' ')' -and -not -iname '*.html' | xargs grep -Pa '\x00' - *.h - *.cpp - *.htm And after verified with: $ find contrib/ -type f -and '(' -iname 'LICENSE*' -or -iname 'COPYING*' -or -iname 'COPYRIGHT*' ')' -and -not '(' -iname '*.html' -or -iname '*.htm' -or -iname '*.rtf' -or -name '*.cpp' -or -name '*.h' -or -iname '*.json' ')' | xargs file -b | sort -u ASCII text ASCII text, with CR line terminators ASCII text, with very long lines empty UTF-8 Unicode text --- utils/list-licenses/list-licenses.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/list-licenses/list-licenses.sh b/utils/list-licenses/list-licenses.sh index 987179e26a8..8eee3f97253 100755 --- a/utils/list-licenses/list-licenses.sh +++ b/utils/list-licenses/list-licenses.sh @@ -7,7 +7,7 @@ ls -1 -d ${LIBS_PATH}/*/ | grep -F -v -- '-cmake' | while read LIB; do LIB_NAME=$(basename $LIB) LIB_LICENSE=$( - LC_ALL=C find "$LIB" -type f -and '(' -iname 'LICENSE*' -or -iname 'COPYING*' -or -iname 'COPYRIGHT*' ')' -and -not -iname '*.html' -printf "%d\t%p\n" | + LC_ALL=C find "$LIB" -type f -and '(' -iname 'LICENSE*' -or -iname 'COPYING*' -or -iname 'COPYRIGHT*' ')' -and -not '(' -iname '*.html' -or -iname '*.htm' -or -iname '*.rtf' -or -name '*.cpp' -or -name '*.h' -or -iname '*.json' ')' -printf "%d\t%p\n" | awk ' BEGIN { IGNORECASE=1; min_depth = 0 } /LICENSE/ { if (!min_depth || $1 <= min_depth) { min_depth = $1; license = $2 } } From 0f4fdcbf389909ed2e642263b0d6a65a3580d8e0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 10 Sep 2020 02:05:41 +0300 Subject: [PATCH 080/625] Pass -fsanitize-blacklist for TSAN only under clang (gcc does not support this) And no such check for -fsnaitize=memory, since gcc does not support it anyway. --- cmake/sanitize.cmake | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 32443ed78c3..7c7e9c388a0 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -36,7 +36,15 @@ if (SANITIZE) endif () elseif (SANITIZE STREQUAL "thread") - set (TSAN_FLAGS "-fsanitize=thread -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/tsan_suppressions.txt") + set (TSAN_FLAGS "-fsanitize=thread") + if (COMPILER_CLANG) + set (TSAN_FLAGS "${TSAN_FLAGS} -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/tsan_suppressions.txt") + else() + message (WARNING "TSAN suppressions was not passed to the compiler (since the compiler is not clang)") + message (WARNING "Use the following command to pass them manually:") + message (WARNING " export TSAN_OPTIONS=\"$TSAN_OPTIONS suppressions=${CMAKE_SOURCE_DIR}/tests/tsan_suppressions.txt\"") + endif() + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${TSAN_FLAGS}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${TSAN_FLAGS}") From b9bf67b6ac31741246dce7790ec3890fd599ff7d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 10 Sep 2020 04:27:36 +0300 Subject: [PATCH 081/625] 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 082/625] 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 083/625] 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 a64473313971bbd3d461d5c7b68165b39d4515e0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Sep 2020 12:05:57 +0300 Subject: [PATCH 084/625] Attempt to make performance test more reliable --- programs/server/Server.cpp | 8 +- src/Common/remapExecutable.cpp | 213 +++++++++++++++++++++++++++++++++ src/Common/remapExecutable.h | 7 ++ 3 files changed, 227 insertions(+), 1 deletion(-) create mode 100644 src/Common/remapExecutable.cpp create mode 100644 src/Common/remapExecutable.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f24ba444203..8149623ffce 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -307,6 +308,11 @@ int Server::main(const std::vector & /*args*/) { if (config().getBool("mlock_executable", false)) { + LOG_DEBUG(log, "Will remap executable in memory."); + remapExecutable(); + LOG_DEBUG(log, "The code in memory has been successfully remapped."); + +/* if (hasLinuxCapability(CAP_IPC_LOCK)) { LOG_TRACE(log, "Will mlockall to prevent executable memory from being paged out. It may take a few seconds."); @@ -321,7 +327,7 @@ int Server::main(const std::vector & /*args*/) " It could happen due to incorrect ClickHouse package installation." " You could resolve the problem manually with 'sudo setcap cap_ipc_lock=+ep {}'." " Note that it will not work on 'nosuid' mounted filesystems.", executable_path); - } + }*/ } } #endif diff --git a/src/Common/remapExecutable.cpp b/src/Common/remapExecutable.cpp new file mode 100644 index 00000000000..f7f353a83c6 --- /dev/null +++ b/src/Common/remapExecutable.cpp @@ -0,0 +1,213 @@ +#if defined(__linux__) && defined(__amd64__) && defined(__SSE2__) + +#include +#include +#include + +#include + +#include + +#include +#include +#include +#include +#include + +#include "remapExecutable.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int CANNOT_ALLOCATE_MEMORY; +} + + +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; +} + + +/** 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"( + movq %%rdi,%%rax; + movq %%rsi,%%rdi; + movq %%rdx,%%rsi; + movq %%rcx,%%rdx; + movq %%r8,%%r10; + movq %%r9,%%r8; + movq 8(%%rsp),%%r9; + syscall; + ret + )" : : : "memory"); + return 0; +} + + +__attribute__((__always_inline__)) void our_memcpy(char * __restrict dst, const char * __restrict src, ssize_t n) +{ + while (n > 0) + { + _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), + _mm_loadu_si128(reinterpret_cast(src))); + + dst += 16; + src += 16; + n -= 16; + } +} + + +__attribute__((__noinline__)) void remapToHugeStep3(void * scratch, size_t size, size_t offset) +{ + /// The function should not use the stack, otherwise various optimizations, including "omit-frame-pointer" may break the code. + + /// Unmap the scratch area. + our_syscall(SYS_munmap, scratch, size); + + /** The return address of this function is pointing to scratch area (because it was called from there). + * But the scratch area no longer exists. We should correct the return address by subtracting the offset. + */ + __asm__ __volatile__("subq %0, 8(%%rsp)" : : "r"(offset) : "memory"); +} + + +__attribute__((__noinline__)) void remapToHugeStep2(void * begin, size_t size, void * scratch) +{ + /** Unmap old memory region with the code of our program. + * Our instruction pointer is located inside scratch area and this function can execute after old code is unmapped. + * But it cannot call any other functions because they are not available at usual addresses + * - that's why we have to use "our_syscall" and "our_memcpy" functions. + * (Relative addressing may continue to work but we should not assume that). + */ + + int64_t offset = reinterpret_cast(scratch) - reinterpret_cast(begin); + int64_t (*syscall_func)(...) = reinterpret_cast(reinterpret_cast(our_syscall) + offset); + + //char dot = '.'; + //syscall_func(SYS_write, 2, &dot, 1); + + int64_t munmap_res = syscall_func(SYS_munmap, begin, size); + if (munmap_res != 0) + return; + + //syscall_func(SYS_write, 2, &dot, 1); + + /// Map new anonymous memory region in place of old region with code. + + int64_t mmap_res = syscall_func(SYS_mmap, begin, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS | MAP_FIXED, -1, 0); + if (-1 == mmap_res) + syscall_func(SYS_exit, 1); + //syscall_func(SYS_write, 2, &dot, 1); + + /// As the memory region is anonymous, we can do madvise with MADV_HUGEPAGE. + + syscall_func(SYS_madvise, begin, size, MADV_HUGEPAGE); + //syscall_func(SYS_write, 2, &dot, 1); + + /// Copy the code from scratch area to the old memory location. + + our_memcpy(reinterpret_cast(begin), reinterpret_cast(scratch), size); + //syscall_func(SYS_write, 2, &dot, 1); + + /// Make the memory area with the code executable and non-writable. + + syscall_func(SYS_mprotect, begin, size, PROT_READ | PROT_EXEC); + //syscall_func(SYS_write, 2, &dot, 1); + + /** Step 3 function should unmap the scratch area. + * The currently executed code is located in the scratch area and cannot be removed here. + * We have to call another function and use its address from the original location (not in scratch area). + * To do it, we obtain it's pointer and call by pointer. + */ + + void(* volatile step3)(void*, size_t, size_t) = remapToHugeStep3; + step3(scratch, size, offset); +} + + +__attribute__((__noinline__)) void remapToHugeStep1(void * begin, size_t size) +{ + /// Allocate scratch area and copy the code there. + + void * scratch = mmap(nullptr, size, PROT_READ | PROT_WRITE | PROT_EXEC, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); + if (MAP_FAILED == scratch) + throwFromErrno(fmt::format("Cannot mmap {} bytes", size), ErrorCodes::CANNOT_ALLOCATE_MEMORY); + + memcpy(scratch, begin, size); + + /// Offset to the scratch area from previous location. + + int64_t offset = reinterpret_cast(scratch) - reinterpret_cast(begin); + + /// Jump to the next function inside the scratch area. + + reinterpret_cast(reinterpret_cast(remapToHugeStep2) + offset)(begin, size, scratch); +} + +} + + +void remapExecutable() +{ + auto [begin, size] = getMappedArea(reinterpret_cast(remapExecutable)); + remapToHugeStep1(begin, size); +} + +} + +#else + +namespace DB +{ + +void remapExecutable() {} + +} + +#endif diff --git a/src/Common/remapExecutable.h b/src/Common/remapExecutable.h new file mode 100644 index 00000000000..7acb61f13bd --- /dev/null +++ b/src/Common/remapExecutable.h @@ -0,0 +1,7 @@ +namespace DB +{ + +/// This function tries to reallocate the code of the running program in a more efficient way. +void remapExecutable(); + +} From 532d121100fc696fa2edb2d3dff863907850c218 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Sep 2020 12:14:31 +0300 Subject: [PATCH 085/625] Fix typo --- src/Common/remapExecutable.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/remapExecutable.cpp b/src/Common/remapExecutable.cpp index f7f353a83c6..ec8b1703b0a 100644 --- a/src/Common/remapExecutable.cpp +++ b/src/Common/remapExecutable.cpp @@ -163,7 +163,7 @@ __attribute__((__noinline__)) void remapToHugeStep2(void * begin, size_t size, v /** Step 3 function should unmap the scratch area. * The currently executed code is located in the scratch area and cannot be removed here. * We have to call another function and use its address from the original location (not in scratch area). - * To do it, we obtain it's pointer and call by pointer. + * To do it, we obtain its pointer and call by pointer. */ void(* volatile step3)(void*, size_t, size_t) = remapToHugeStep3; From 338b69201d126a9ccc0be828b6fed4f6a0affef9 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 10 Sep 2020 17:20:28 +0800 Subject: [PATCH 086/625] 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 087/625] 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 088/625] 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 65e9c678f8990756ac22ad2ad10bc08a9c0ec4b4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Sep 2020 17:47:02 +0300 Subject: [PATCH 089/625] Disable under certain conditions --- src/Common/remapExecutable.cpp | 2 +- src/Common/ya.make | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/remapExecutable.cpp b/src/Common/remapExecutable.cpp index ec8b1703b0a..b41fece0c79 100644 --- a/src/Common/remapExecutable.cpp +++ b/src/Common/remapExecutable.cpp @@ -1,4 +1,4 @@ -#if defined(__linux__) && defined(__amd64__) && defined(__SSE2__) +#if defined(__linux__) && defined(__amd64__) && defined(__SSE2__) && !defined(SANITIZER) && defined(NDEBUG) #include #include diff --git a/src/Common/ya.make b/src/Common/ya.make index d9a7a2ce4de..72f1fa42756 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -74,6 +74,7 @@ SRCS( QueryProfiler.cpp quoteString.cpp randomSeed.cpp + remapExecutable.cpp RemoteHostFilter.cpp renameat2.cpp RWLock.cpp From ca2a33008b291bc5d1507b568ac31d588a6aa3d8 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 2 Sep 2020 19:42:24 +0300 Subject: [PATCH 090/625] faster --- docker/test/performance-comparison/eqmed.sql | 4 ++-- docker/test/performance-comparison/perf.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/performance-comparison/eqmed.sql b/docker/test/performance-comparison/eqmed.sql index f7f8d6ac40d..139f0758798 100644 --- a/docker/test/performance-comparison/eqmed.sql +++ b/docker/test/performance-comparison/eqmed.sql @@ -8,7 +8,7 @@ select from ( -- quantiles of randomization distributions - select quantileExactForEach(0.999)( + select quantileExactForEach(0.99)( arrayMap(x, y -> abs(x - y), metrics_by_label[1], metrics_by_label[2]) as d ) threshold ---- uncomment to see what the distribution is really like @@ -33,7 +33,7 @@ from -- strip the query away before the join -- it might be several kB long; (select metrics, run, version from table) no_query, -- duplicate input measurements into many virtual runs - numbers(1, 100000) nn + numbers(1, 10000) nn -- for each virtual run, randomly reorder measurements order by virtual_run, rand() ) virtual_runs diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index e1476d9aeb4..05e89c9e44c 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -20,7 +20,7 @@ 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=int(os.environ.get('CHPC_RUNS', 7)), help='Number of query runs per server. Defaults to CHPC_RUNS environment variable.') 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.') From 26348ad0143f881c8d14e41e0c80d706614ab110 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 10 Sep 2020 18:48:39 +0300 Subject: [PATCH 091/625] fixup --- docker/test/performance-comparison/report.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 1003a6d0e1a..b3f8ef01138 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -372,7 +372,7 @@ if args.report == 'main': 'New, s', # 1 'Ratio of speedup (-) or slowdown (+)', # 2 'Relative difference (new − old) / old', # 3 - 'p < 0.001 threshold', # 4 + 'p < 0.01 threshold', # 4 # Failed # 5 'Test', # 6 '#', # 7 @@ -416,7 +416,7 @@ if args.report == 'main': 'Old, s', #0 'New, s', #1 'Relative difference (new - old)/old', #2 - 'p < 0.001 threshold', #3 + 'p < 0.01 threshold', #3 # Failed #4 'Test', #5 '#', #6 @@ -649,7 +649,7 @@ elif args.report == 'all-queries': 'New, s', #3 'Ratio of speedup (-) or slowdown (+)', #4 'Relative difference (new − old) / old', #5 - 'p < 0.001 threshold', #6 + 'p < 0.01 threshold', #6 'Test', #7 '#', #8 'Query', #9 From 513e0043f73c51e67454a5dedbe8d9f65d85da72 Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 10 Sep 2020 18:57:18 +0300 Subject: [PATCH 092/625] initial docs header --- cmake_files_header.md | 137 ++++++++++++++++++++++++++++++++++++ cmake_flags_md_generator.sh | 1 + 2 files changed, 138 insertions(+) create mode 100644 cmake_files_header.md create mode 100644 cmake_flags_md_generator.sh diff --git a/cmake_files_header.md b/cmake_files_header.md new file mode 100644 index 00000000000..bb613edaf1b --- /dev/null +++ b/cmake_files_header.md @@ -0,0 +1,137 @@ +# CMake flags + +## Developer's guide for adding new CMake options + +### Don't be obvious. Be informative. + +Bad: +``` +option (ENABLE_TESTS "Enables testing" OFF) +``` + +This description is quite useless as is neither gives the viewer any additional information nor explains the option +purpose. If the option's name is quite self-descriptive, prefer the empty description. + +Better: + +``` +option(ENABLE_TESTS OFF) +``` + +If the option's purpose can't be guessed by its name, or the purpose guess may be misleading, leave a comment above +the `option()` line and explain what it does. The best way would be linking the docs page (if it exists). +The comment is parsed into a separate column (see below). + +Even better: + +``` +# Adds the ability to test ClickHouse using Google.Test (would produce another target unit_tests_dbms). +# see tests/CMakeLists.txt for implementation detail. +option(ENABLE_GTEST_TESTS OFF) +``` + +### If the option's state could produce unwanted (or unusual) result, explicitly warn the user. + +Suppose you have an option that may strip debug symbols from the ClickHouse's part. +This can speed up the linking process, but produces a binary that cannot be debugged. +In that case, prefer explicitly raising a warning telling the developer that he may be doing something wrong. +Also, such options should be disabled if applies. + +Bad: +``` +option(STRIP_DEBUG_SYMBOLS_FUNCTIONS + "Do not generate debugger info for ClickHouse functions. + ${STRIP_DSF_DEFAULT}) + +if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) + target_compile_options(clickhouse_functions PRIVATE "-g0") +endif() + +``` +Better: + +``` +# Provides faster linking and lower binary size. +# Tradeoff is the inability to debug some source files with e.g. gdb +# (empty stack frames and no local variables)." +option(STRIP_DEBUG_SYMBOLS_FUNCTIONS + "Do not generate debugger info for ClickHouse functions." + ${STRIP_DSF_DEFAULT}) + +if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) + message(WARNING "Not generating debugger info for ClickHouse functions") + target_compile_options(clickhouse_functions PRIVATE "-g0") +endif() +``` + +### In the option's description, explain WHAT the option does rather than WHY it does something. + +The WHY explanation should be placed in the comment. +You may find that the option's name is self-descriptive. + +Bad: + +``` +option(ENABLE_THINLTO "Enable Thin LTO. Only applicable for clang. It's also suppressed when building with tests or sanitizers." ON) +``` + +Better: + +``` +# Only applicable for clang. +# Turned off when building with tests or sanitizers. +option(ENABLE_THINLTO ON). +``` + +### Don't assume other developers know as much as you do. + +In ClickHouse, there are many tools used that an ordinary developer may not know. If you are in doubt, give a link to +the tool's docs. It won't take much of your time. + +Bad: + +``` +option(ENABLE_THINLTO "Enable Thin LTO. Only applicable for clang. It's also suppressed when building with tests or sanitizers." ON) +``` + +Better (combined with the above hint): + +``` +# https://clang.llvm.org/docs/ThinLTO.html +# Only applicable for clang. +# Turned off when building with tests or sanitizers. +option(ENABLE_THINLTO ON). +``` + +Other example, bad: + +``` +option (USE_INCLUDE_WHAT_YOU_USE "Use 'include-what-you-use' tool" OFF) +``` + +Better: + +``` +# https://github.com/include-what-you-use/include-what-you-use +option (USE_INCLUDE_WHAT_YOU_USE OFF) +``` + +### Prefer consistent default values. + +CMake allows you to pass a plethora of values representing boolean `true/false`, e.g. `1, ON, YES, ...`. +Prefer the `ON/OFF` values, if possible. + + +## List of CMake flags + +* This list is auto-generated by [this bash script](bash.sh). +* The flag name is a link to its position in the code. + +| Name | Description | Default value | Comment | +|------|-------------|---------------|---------| +| \#0 | 89354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | +| \#1 | 90329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | +| \#2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | +| \#N | … | … | … | … | … | + + diff --git a/cmake_flags_md_generator.sh b/cmake_flags_md_generator.sh new file mode 100644 index 00000000000..e72f7660e6d --- /dev/null +++ b/cmake_flags_md_generator.sh @@ -0,0 +1 @@ +#!/bin/bash/ From 45340c701dc517b29db5a1047c306f88ba891722 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 10 Sep 2020 19:49:57 +0300 Subject: [PATCH 093/625] changelog for 20.8 --- CHANGELOG.md | 148 ++++++++++++++++++++++ utils/simple-backport/backport.sh | 7 +- utils/simple-backport/format-changelog.py | 2 +- 3 files changed, 154 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 345ee2c6213..f3266520eb1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,151 @@ +## ClickHouse release 20.8 + +### ClickHouse release v20.8.2.3-stable, 2020-09-08 + +#### Backward Incompatible Change + +* Now `OPTIMIZE FINAL` query doesn't recalculate TTL for parts that were added before TTL was created. Use `ALTER TABLE ... MATERIALIZE TTL` once to calculate them, after that `OPTIMIZE FINAL` will evaluate TTL's properly. This behavior never worked for replicated tables. [#14220](https://github.com/ClickHouse/ClickHouse/pull/14220) ([alesapin](https://github.com/alesapin)). +* Extend `parallel_distributed_insert_select` setting, adding an option to run `INSERT` into local table. The setting changes type from `Bool` to `UInt64`, so the values `false` and `true` are no longer supported. If you have these values in server configuration, the server will not start. Please replace them with `0` and `1`, respectively. [#14060](https://github.com/ClickHouse/ClickHouse/pull/14060) ([Azat Khuzhin](https://github.com/azat)). +* Remove support for the `ODBCDriver` input/output format. This was a deprecated format once used for communication with the ClickHouse ODBC driver, now long superseded by the `ODBCDriver2` format. Resolves [#13629](https://github.com/ClickHouse/ClickHouse/issues/13629). [#13847](https://github.com/ClickHouse/ClickHouse/pull/13847) ([hexiaoting](https://github.com/hexiaoting)). + +#### New Feature + +* Add `countDigits(x)` function that count number of decimal digits in integer or decimal column. Add `isDecimalOverflow(d, [p])` function that checks if the value in Decimal column is out of its (or specified) precision. [#14151](https://github.com/ClickHouse/ClickHouse/pull/14151) ([Artem Zuikov](https://github.com/4ertus2)). +* Add setting `min_index_granularity_bytes` that protects against accidentally creating a table with very low `index_granularity_bytes` setting. [#14139](https://github.com/ClickHouse/ClickHouse/pull/14139) ([Bharat Nallan](https://github.com/bharatnc)). +* Add the ability to specify `Default` compression codec for columns that correspond to settings specified in `config.xml`. Implements: [#9074](https://github.com/ClickHouse/ClickHouse/issues/9074). [#14049](https://github.com/ClickHouse/ClickHouse/pull/14049) ([alesapin](https://github.com/alesapin)). +* Added `date_trunc` function that truncates a date/time value to a specified date/time part. [#13888](https://github.com/ClickHouse/ClickHouse/pull/13888) ([Vladimir Golovchenko](https://github.com/vladimir-golovchenko)). +* Add `time_zones` table. [#13880](https://github.com/ClickHouse/ClickHouse/pull/13880) ([Bharat Nallan](https://github.com/bharatnc)). +* Add function `defaultValueOfTypeName` that returns the default value for a given type. [#13877](https://github.com/ClickHouse/ClickHouse/pull/13877) ([hcz](https://github.com/hczhcz)). +* Add `quantileExactLow` and `quantileExactHigh` implementations with respective aliases for `medianExactLow` and `medianExactHigh`. [#13818](https://github.com/ClickHouse/ClickHouse/pull/13818) ([Bharat Nallan](https://github.com/bharatnc)). +* Add function `normalizeQuery` that replaces literals, sequences of literals and complex aliases with placeholders. Add function `normalizedQueryHash` that returns identical 64bit hash values for similar queries. It helps to analyze query log. This closes [#11271](https://github.com/ClickHouse/ClickHouse/issues/11271). [#13816](https://github.com/ClickHouse/ClickHouse/pull/13816) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add new optional section to the main config. [#13425](https://github.com/ClickHouse/ClickHouse/pull/13425) ([Vitaly Baranov](https://github.com/vitlibar)). +* Add `ALTER SAMPLE BY` statement that allows to change table sample clause. [#13280](https://github.com/ClickHouse/ClickHouse/pull/13280) ([Amos Bird](https://github.com/amosbird)). +* Function `position` now supports optional `start_pos` argument. [#13237](https://github.com/ClickHouse/ClickHouse/pull/13237) ([vdimir](https://github.com/vdimir)). +* Add types `Int128`, `Int256`, `UInt256` and related functions for them. Extend Decimals with Decimal256 (precision up to 76 digits). New types are under the setting `allow_experimental_bigint_types`. [#13097](https://github.com/ClickHouse/ClickHouse/pull/13097) ([Artem Zuikov](https://github.com/4ertus2)). +* Support Kerberos authentication in Kafka, using `krb5` and `cyrus-sasl` libraries. [#12771](https://github.com/ClickHouse/ClickHouse/pull/12771) ([Ilya Golshtein](https://github.com/ilejn)). +* Support `MaterializeMySQL` database engine. Implements [#4006](https://github.com/ClickHouse/ClickHouse/issues/4006). [#10851](https://github.com/ClickHouse/ClickHouse/pull/10851) ([Winter Zhang](https://github.com/zhang2014)). + +#### Bug Fix + +* Check for array size overflow in `topK` aggregate function. Without this check the user may send a query with carefully crafter parameters that will lead to server crash. This closes [#14452](https://github.com/ClickHouse/ClickHouse/issues/14452). [#14467](https://github.com/ClickHouse/ClickHouse/pull/14467) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bug which leads to wrong merges assignment if table has partitions with a single part. [#14444](https://github.com/ClickHouse/ClickHouse/pull/14444) ([alesapin](https://github.com/alesapin)). +* Stop query execution if exception happened in `PipelineExecutor` itself. This could prevent rare possible query hung. Continuation of [#14334](https://github.com/ClickHouse/ClickHouse/issues/14334). [#14402](https://github.com/ClickHouse/ClickHouse/pull/14402) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Stop query execution if exception happened in `PipelineExecutor` itself. This could prevent rare possible query hung. [#14334](https://github.com/ClickHouse/ClickHouse/pull/14334) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix crash during `ALTER` query for table which was created `AS table_function`. Fixes [#14212](https://github.com/ClickHouse/ClickHouse/issues/14212). [#14326](https://github.com/ClickHouse/ClickHouse/pull/14326) ([alesapin](https://github.com/alesapin)). +* Fix exception during ALTER LIVE VIEW query with REFRESH command. [#14320](https://github.com/ClickHouse/ClickHouse/pull/14320) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix QueryPlan lifetime (for EXPLAIN PIPELINE graph=1) for queries with nested interpreter. [#14315](https://github.com/ClickHouse/ClickHouse/pull/14315) ([Azat Khuzhin](https://github.com/azat)). +* Fix segfault in `clickhouse-odbc-bridge` during schema fetch from some external sources. This PR fixes https://github.com/ClickHouse/ClickHouse/issues/13861. [#14267](https://github.com/ClickHouse/ClickHouse/pull/14267) ([Vitaly Baranov](https://github.com/vitlibar)). +* Disallows `CODEC` on `ALIAS` column type. Fixes [#13911](https://github.com/ClickHouse/ClickHouse/issues/13911). [#14263](https://github.com/ClickHouse/ClickHouse/pull/14263) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix handling of empty transactions in `MaterializeMySQL` database engine. This fixes [#14235](https://github.com/ClickHouse/ClickHouse/issues/14235). [#14253](https://github.com/ClickHouse/ClickHouse/pull/14253) ([BohuTANG](https://github.com/BohuTANG)). +* fixes [#14231](https://github.com/ClickHouse/ClickHouse/issues/14231) fix wrong lexer in MaterializeMySQL database engine dump stage. [#14232](https://github.com/ClickHouse/ClickHouse/pull/14232) ([Winter Zhang](https://github.com/zhang2014)). +* Fix crash in mark inclusion search introduced in https://github.com/ClickHouse/ClickHouse/pull/12277 . [#14225](https://github.com/ClickHouse/ClickHouse/pull/14225) ([Amos Bird](https://github.com/amosbird)). +* Fixed incorrect sorting order if LowCardinality column. This fixes [#13958](https://github.com/ClickHouse/ClickHouse/issues/13958). [#14223](https://github.com/ClickHouse/ClickHouse/pull/14223) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix creation of tables with named tuples. This fixes [#13027](https://github.com/ClickHouse/ClickHouse/issues/13027). [#14143](https://github.com/ClickHouse/ClickHouse/pull/14143) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix formatting of minimal negative decimal numbers. This fixes https://github.com/ClickHouse/ClickHouse/issues/14111. [#14119](https://github.com/ClickHouse/ClickHouse/pull/14119) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* When waiting for a dictionary update to complete, use the timeout specified by `query_wait_timeout_milliseconds` setting instead of a hard-coded value. [#14105](https://github.com/ClickHouse/ClickHouse/pull/14105) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix DistributedFilesToInsert metric (zeroed when it should not). [#14095](https://github.com/ClickHouse/ClickHouse/pull/14095) ([Azat Khuzhin](https://github.com/azat)). +* Fix pointInPolygon with const 2d array as polygon. [#14079](https://github.com/ClickHouse/ClickHouse/pull/14079) ([Alexey Ilyukhov](https://github.com/livace)). +* Fixed wrong mount point in extra info for `Poco::Exception: no space left on device`. [#14050](https://github.com/ClickHouse/ClickHouse/pull/14050) ([tavplubix](https://github.com/tavplubix)). +* Fix GRANT ALL statement when executed on a non-global level. [#13987](https://github.com/ClickHouse/ClickHouse/pull/13987) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix parser to reject create table as table function with engine. [#13940](https://github.com/ClickHouse/ClickHouse/pull/13940) ([hcz](https://github.com/hczhcz)). +* Fix wrong results in select queries with `DISTINCT` keyword in case `optimize_duplicate_order_by_and_distinct` setting is enabled. [#13925](https://github.com/ClickHouse/ClickHouse/pull/13925) ([Artem Zuikov](https://github.com/4ertus2)). +* Fixed potential deadlock when renaming `Distributed` table. [#13922](https://github.com/ClickHouse/ClickHouse/pull/13922) ([tavplubix](https://github.com/tavplubix)). +* Fix incorrect sorting for `FixedString` columns. Fixes [#13182](https://github.com/ClickHouse/ClickHouse/issues/13182). [#13887](https://github.com/ClickHouse/ClickHouse/pull/13887) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix topK/topKWeighted merge (with non-default parameters). [#13817](https://github.com/ClickHouse/ClickHouse/pull/13817) ([Azat Khuzhin](https://github.com/azat)). +* Fix reading from MergeTree table with INDEX of type SET fails when comparing against NULL. This fixes [#13686](https://github.com/ClickHouse/ClickHouse/issues/13686). [#13793](https://github.com/ClickHouse/ClickHouse/pull/13793) ([Amos Bird](https://github.com/amosbird)). +* Fix arrayJoin() capturing in lambda (LOGICAL_ERROR). [#13792](https://github.com/ClickHouse/ClickHouse/pull/13792) ([Azat Khuzhin](https://github.com/azat)). +* Fix step overflow in range(). [#13790](https://github.com/ClickHouse/ClickHouse/pull/13790) ([Azat Khuzhin](https://github.com/azat)). +* Fixed `Directory not empty` error when concurrently executing `DROP DATABASE` and `CREATE TABLE`. [#13756](https://github.com/ClickHouse/ClickHouse/pull/13756) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add range check for h3KRing function. This fixes [#13633](https://github.com/ClickHouse/ClickHouse/issues/13633). [#13752](https://github.com/ClickHouse/ClickHouse/pull/13752) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix race condition between DETACH and background merges. Parts may revive after detach. This is continuation of [#8602](https://github.com/ClickHouse/ClickHouse/issues/8602) that did not fix the issue but introduced a test that started to fail in very rare cases, demonstrating the issue. [#13746](https://github.com/ClickHouse/ClickHouse/pull/13746) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix logging Settings.Names/Values when log_queries_min_type > QUERY_START. [#13737](https://github.com/ClickHouse/ClickHouse/pull/13737) ([Azat Khuzhin](https://github.com/azat)). +* Fixes /replicas_status endpoint response status code when verbose=1. [#13722](https://github.com/ClickHouse/ClickHouse/pull/13722) ([javi santana](https://github.com/javisantana)). +* Fix incorrect message in `clickhouse-server.init` while checking user and group. [#13711](https://github.com/ClickHouse/ClickHouse/pull/13711) ([ylchou](https://github.com/ylchou)). +* Fix visible data clobbering by progress bar in client in interactive mode. This fixes [#12562](https://github.com/ClickHouse/ClickHouse/issues/12562) and [#13369](https://github.com/ClickHouse/ClickHouse/issues/13369) and [#13584](https://github.com/ClickHouse/ClickHouse/issues/13584) and fixes [#12964](https://github.com/ClickHouse/ClickHouse/issues/12964). [#13691](https://github.com/ClickHouse/ClickHouse/pull/13691) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Do not optimize any(arrayJoin()) -> arrayJoin() under optimize_move_functions_out_of_any. [#13681](https://github.com/ClickHouse/ClickHouse/pull/13681) ([Azat Khuzhin](https://github.com/azat)). +* Fix crash in JOIN with StorageMerge and `set enable_optimize_predicate_expression=1`. [#13679](https://github.com/ClickHouse/ClickHouse/pull/13679) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix typo in error message about `The value of 'number_of_free_entries_in_pool_to_lower_max_size_of_merge' setting`. [#13678](https://github.com/ClickHouse/ClickHouse/pull/13678) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Concurrent `ALTER ... REPLACE/MOVE PARTITION ...` queries might cause deadlock. It's fixed. [#13626](https://github.com/ClickHouse/ClickHouse/pull/13626) ([tavplubix](https://github.com/tavplubix)). +* Fixed the behaviour when sometimes cache-dictionary returned default value instead of present value from source. [#13624](https://github.com/ClickHouse/ClickHouse/pull/13624) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix secondary indices corruption in compact parts. [#13538](https://github.com/ClickHouse/ClickHouse/pull/13538) ([Anton Popov](https://github.com/CurtizJ)). +* Fix premature `ON CLUSTER` timeouts for queries that must be executed on a single replica. Fixes [#6704](https://github.com/ClickHouse/ClickHouse/issues/6704), [#7228](https://github.com/ClickHouse/ClickHouse/issues/7228), [#13361](https://github.com/ClickHouse/ClickHouse/issues/13361), [#11884](https://github.com/ClickHouse/ClickHouse/issues/11884). [#13450](https://github.com/ClickHouse/ClickHouse/pull/13450) ([alesapin](https://github.com/alesapin)). +* Fix wrong code in function `netloc`. This fixes [#13335](https://github.com/ClickHouse/ClickHouse/issues/13335). [#13446](https://github.com/ClickHouse/ClickHouse/pull/13446) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix possible race in `StorageMemory`. https://clickhouse-test-reports.s3.yandex.net/0/9cac8a7244063d2092ad25d45502611e18d3749c/stress_test_(thread)/stderr.log Have no idea how to write a test. [#13416](https://github.com/ClickHouse/ClickHouse/pull/13416) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix missing or excessive headers in `TSV/CSVWithNames` formats. This fixes [#12504](https://github.com/ClickHouse/ClickHouse/issues/12504). [#13343](https://github.com/ClickHouse/ClickHouse/pull/13343) ([Azat Khuzhin](https://github.com/azat)). +* Fix parsing row policies from users.xml when names of databases or tables contain dots. This fixes https://github.com/ClickHouse/ClickHouse/issues/5779, https://github.com/ClickHouse/ClickHouse/issues/12527. [#13199](https://github.com/ClickHouse/ClickHouse/pull/13199) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix access to redis dictionary after connection was dropped once. It may happen with `cache` and `direct` dictionary layouts. [#13082](https://github.com/ClickHouse/ClickHouse/pull/13082) ([Anton Popov](https://github.com/CurtizJ)). +* Removed wrong auth access check when using ClickHouseDictionarySource to query remote tables. [#12756](https://github.com/ClickHouse/ClickHouse/pull/12756) ([sundyli](https://github.com/sundy-li)). +* subquery hash values are not enough to distinguish. https://github.com/ClickHouse/ClickHouse/issues/8333. [#8367](https://github.com/ClickHouse/ClickHouse/pull/8367) ([Amos Bird](https://github.com/amosbird)). + +#### Improvement + +* Now it's possible to `ALTER TABLE table_name FETCH PARTITION partition_expr FROM 'zk://:/path-in-zookeeper'`. It's useful for shipping data to new clusters. [#14155](https://github.com/ClickHouse/ClickHouse/pull/14155) ([Amos Bird](https://github.com/amosbird)). +* Slightly better performance of Memory table if it was constructed from a huge number of very small blocks (that's unlikely). Author of the idea: [Mark Papadakis](https://github.com/markpapadakis). Closes [#14043](https://github.com/ClickHouse/ClickHouse/issues/14043). [#14056](https://github.com/ClickHouse/ClickHouse/pull/14056) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Conditional aggregate functions (for example: `avgIf`, `sumIf`, `maxIf`) should return `NULL` when miss rows and use nullable arguments. [#13964](https://github.com/ClickHouse/ClickHouse/pull/13964) ([Winter Zhang](https://github.com/zhang2014)). +* Increase limit in -Resample combinator to 1M. [#13947](https://github.com/ClickHouse/ClickHouse/pull/13947) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Corrected an error in AvroConfluent format that caused the Kafka table engine to stop processing messages when an abnormally small, malformed, message was received. [#13941](https://github.com/ClickHouse/ClickHouse/pull/13941) ([Gervasio Varela](https://github.com/gervarela)). +* Fix wrong error for long queries. It was possible to get syntax error other than `Max query size exceeded` for correct query. [#13928](https://github.com/ClickHouse/ClickHouse/pull/13928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Better error message for null value of TabSeparatedRow format. [#13906](https://github.com/ClickHouse/ClickHouse/pull/13906) ([jiang tao](https://github.com/tomjiang1987)). +* Function `arrayCompact` will compare NaNs bitwise if the type of array elements is Float32/Float64. In previous versions NaNs were always not equal if the type of array elements is Float32/Float64 and were always equal if the type is more complex, like Nullable(Float64). This closes [#13857](https://github.com/ClickHouse/ClickHouse/issues/13857). [#13868](https://github.com/ClickHouse/ClickHouse/pull/13868) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix data race in `lgamma` function. This race was caught only in `tsan`, no side effects a really happened. [#13842](https://github.com/ClickHouse/ClickHouse/pull/13842) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* 1. Add [GTID-Based Replication](https://dev.mysql.com/doc/refman/5.7/en/replication-gtids-concepts.html), it works even when replication topology changes, and supported/prefered in MySQL 5.6/5.7/8.0 2. Add BIT/SET filed type supports 3. Fix up varchar type meta length bug. [#13820](https://github.com/ClickHouse/ClickHouse/pull/13820) ([BohuTANG](https://github.com/BohuTANG)). +* Avoid too slow queries when arrays are manipulated as fields. Throw exception instead. [#13753](https://github.com/ClickHouse/ClickHouse/pull/13753) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added Redis requirepass authorization. [#13688](https://github.com/ClickHouse/ClickHouse/pull/13688) ([Ivan Torgashov](https://github.com/it1804)). +* Add MergeTree Write-Ahead-Log(WAL) dump tool. [#13640](https://github.com/ClickHouse/ClickHouse/pull/13640) ([BohuTANG](https://github.com/BohuTANG)). +* In previous versions `lcm` function may produce assertion violation in debug build if called with specifically crafted arguments. This fixes [#13368](https://github.com/ClickHouse/ClickHouse/issues/13368). [#13510](https://github.com/ClickHouse/ClickHouse/pull/13510) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Provide monotonicity for `toDate/toDateTime` functions in more cases. Now the input arguments are saturated more naturally and provides better monotonicity. [#13497](https://github.com/ClickHouse/ClickHouse/pull/13497) ([Amos Bird](https://github.com/amosbird)). +* Support compound identifiers for custom settings. [#13496](https://github.com/ClickHouse/ClickHouse/pull/13496) ([Vitaly Baranov](https://github.com/vitlibar)). +* Move parts from DIskLocal to DiskS3 in parallel. [#13459](https://github.com/ClickHouse/ClickHouse/pull/13459) ([Pavel Kovalenko](https://github.com/Jokser)). +* Enable mixed granularity parts by default. [#13449](https://github.com/ClickHouse/ClickHouse/pull/13449) ([alesapin](https://github.com/alesapin)). +* Proper remote host checking in S3 redirects (security-related thing). [#13404](https://github.com/ClickHouse/ClickHouse/pull/13404) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Add QueryTimeMicroseconds, SelectQueryTimeMicroseconds and InsertQueryTimeMicroseconds to system.events. [#13336](https://github.com/ClickHouse/ClickHouse/pull/13336) ([ianton-ru](https://github.com/ianton-ru)). +* Fix assert when decimal has too large negative exponent. Fixes [#13188](https://github.com/ClickHouse/ClickHouse/issues/13188). [#13228](https://github.com/ClickHouse/ClickHouse/pull/13228) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added cache layer for DiskS3 (cache to local disk mark and index files). [#13076](https://github.com/ClickHouse/ClickHouse/pull/13076) ([Pavel Kovalenko](https://github.com/Jokser)). + +#### Performance Improvement + +* Slightly optimize very short queries with LowCardinality. [#14129](https://github.com/ClickHouse/ClickHouse/pull/14129) ([Anton Popov](https://github.com/CurtizJ)). +* Enable parallel INSERTs for table engines `Null`, `Memory`, `Distributed` and `Buffer`. [#14120](https://github.com/ClickHouse/ClickHouse/pull/14120) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fail fast if `max_rows_to_read` limit is exceeded on parts scan. The motivation behind this change is to skip ranges scan for all selected parts if it is clear that `max_rows_to_read` is already exceeded. The change is quite noticeable for queries over big number of parts. [#13677](https://github.com/ClickHouse/ClickHouse/pull/13677) ([Roman Khavronenko](https://github.com/hagen1778)). +* Slightly improve performance of aggregation by UInt8/UInt16 keys. [#13099](https://github.com/ClickHouse/ClickHouse/pull/13099) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Optimize `has()`, `indexOf()` and `countEqual()` functions for `Array(LowCardinality(T))` and constant right arguments. [#12550](https://github.com/ClickHouse/ClickHouse/pull/12550) ([myrrc](https://github.com/myrrc)). +* When performing trivial `INSERT SELECT` queries, automatically set `max_threads` to 1 or `max_insert_threads`, and set `max_block_size` to `min_insert_block_size_rows`. Related to [#5907](https://github.com/ClickHouse/ClickHouse/issues/5907). [#12195](https://github.com/ClickHouse/ClickHouse/pull/12195) ([flynn](https://github.com/ucasFL)). + +#### Build/Testing/Packaging Improvement + +* Actually there are no symlinks there, so `-type f` is enough ``` ~/workspace/ClickHouse/contrib/cctz/testdata/zoneinfo$ find . -type l -ls | wc -l 0 ``` Closes [#14209](https://github.com/ClickHouse/ClickHouse/issues/14209). [#14215](https://github.com/ClickHouse/ClickHouse/pull/14215) ([filimonov](https://github.com/filimonov)). +* Switch tests docker images to use test-base parent. [#14167](https://github.com/ClickHouse/ClickHouse/pull/14167) ([Ilya Yatsishin](https://github.com/qoega)). +* Add the ability to write js-style comments in skip_list.json. [#14159](https://github.com/ClickHouse/ClickHouse/pull/14159) ([alesapin](https://github.com/alesapin)). +* * Adding retry logic when bringing up docker-compose cluster * Increasing COMPOSE_HTTP_TIMEOUT. [#14112](https://github.com/ClickHouse/ClickHouse/pull/14112) ([vzakaznikov](https://github.com/vzakaznikov)). +* Enabled text-log in stress test to find more bugs. [#13855](https://github.com/ClickHouse/ClickHouse/pull/13855) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Testflows LDAP module: adding missing certificates and dhparam.pem for openldap4. [#13780](https://github.com/ClickHouse/ClickHouse/pull/13780) ([vzakaznikov](https://github.com/vzakaznikov)). +* ZooKeeper cannot work reliably in unit tests in CI infrastructure. Using unit tests for ZooKeeper interaction with real ZooKeeper is bad idea from the start (unit tests are not supposed to verify complex distributed systems). We already using integration tests for this purpose and they are better suited. [#13745](https://github.com/ClickHouse/ClickHouse/pull/13745) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added docker image for style check. Added style check that all docker and docker compose files are located in docker directory. [#13724](https://github.com/ClickHouse/ClickHouse/pull/13724) ([Ilya Yatsishin](https://github.com/qoega)). +* FIx cassandra build on Mac OS. [#13708](https://github.com/ClickHouse/ClickHouse/pull/13708) ([Ilya Yatsishin](https://github.com/qoega)). +* Fix link error in shared build. [#13700](https://github.com/ClickHouse/ClickHouse/pull/13700) ([Amos Bird](https://github.com/amosbird)). +* Add a CMake option to fail configuration instead of auto-reconfiguration, enabled by default. [#13687](https://github.com/ClickHouse/ClickHouse/pull/13687) ([Konstantin](https://github.com/podshumok)). +* Updating LDAP user authentication suite to check that it works with RBAC. [#13656](https://github.com/ClickHouse/ClickHouse/pull/13656) ([vzakaznikov](https://github.com/vzakaznikov)). +* Expose version of embedded tzdata via TZDATA_VERSION in system.build_options. [#13648](https://github.com/ClickHouse/ClickHouse/pull/13648) ([filimonov](https://github.com/filimonov)). +* Removed `-DENABLE_CURL_CLIENT` for `contrib/aws`. [#13628](https://github.com/ClickHouse/ClickHouse/pull/13628) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Build ClickHouse with the most fresh tzdata from package repository. [#13623](https://github.com/ClickHouse/ClickHouse/pull/13623) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Increasing health-check timeouts for ClickHouse nodes and adding support to dump docker-compose logs if unhealthy containers found. [#13612](https://github.com/ClickHouse/ClickHouse/pull/13612) ([vzakaznikov](https://github.com/vzakaznikov)). +* Make sure https://github.com/ClickHouse/ClickHouse/issues/10977 is invalid. [#13539](https://github.com/ClickHouse/ClickHouse/pull/13539) ([Amos Bird](https://github.com/amosbird)). +* Enable Shellcheck in CI as a linter of .sh tests. This closes [#13168](https://github.com/ClickHouse/ClickHouse/issues/13168). [#13530](https://github.com/ClickHouse/ClickHouse/pull/13530) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix the remaining shellcheck notices. A preparation to enable Shellcheck. [#13529](https://github.com/ClickHouse/ClickHouse/pull/13529) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added `clickhouse install` script, that is useful if you only have a single binary. [#13528](https://github.com/ClickHouse/ClickHouse/pull/13528) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow to run `clickhouse` binary without configuration. [#13515](https://github.com/ClickHouse/ClickHouse/pull/13515) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Ensure that there is no copy-pasted GPL code. [#13514](https://github.com/ClickHouse/ClickHouse/pull/13514) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Enable check for typos in code with `codespell`. [#13513](https://github.com/ClickHouse/ClickHouse/pull/13513) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix typos in code with codespell. [#13511](https://github.com/ClickHouse/ClickHouse/pull/13511) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Skip PR's from robot-clickhouse. [#13489](https://github.com/ClickHouse/ClickHouse/pull/13489) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Move Dockerfiles from integration tests to `docker/test` directory. docker_compose files are available in `runner` docker container. Docker images are built in CI and not in integration tests. [#13448](https://github.com/ClickHouse/ClickHouse/pull/13448) ([Ilya Yatsishin](https://github.com/qoega)). + +#### Other + +* Create `system` database with `Atomic` engine by default. [#13680](https://github.com/ClickHouse/ClickHouse/pull/13680) ([tavplubix](https://github.com/tavplubix)). +* Fix readline so it dumps history to file now. [#13600](https://github.com/ClickHouse/ClickHouse/pull/13600) ([Amos Bird](https://github.com/amosbird)). + + ## ClickHouse release 20.7 ### ClickHouse release v20.7.2.30-stable, 2020-08-31 diff --git a/utils/simple-backport/backport.sh b/utils/simple-backport/backport.sh index 71920304d56..64f8e6004bf 100755 --- a/utils/simple-backport/backport.sh +++ b/utils/simple-backport/backport.sh @@ -4,7 +4,10 @@ set -e branch="$1" merge_base=$(git merge-base origin/master "origin/$branch") master_git_cmd=(git log "$merge_base..origin/master" --first-parent) -branch_git_cmd=(git log "$merge_base..origin/$branch" --first-parent) +# The history in back branches shouldn't be too crazy, and sometimes we have a PR +# that merges several backport commits there (3f2cba6824fddf31c30bde8c6f4f860572f4f580), +# so don't use --first-parent +branch_git_cmd=(git log "$merge_base..origin/$branch") # Make lists of PRs that were merged into each branch. Use first parent here, or else # we'll get weird things like seeing older master that was merged into a PR branch @@ -30,7 +33,7 @@ fi # Search for PR numbers in commit messages. First variant is normal merge, and second # variant is squashed. Next are some backport message variants. find_prs=(sed -n "s/^.*merg[eding]*.*#\([[:digit:]]\+\).*$/\1/Ip; - s/^.*(#\([[:digit:]]\+\))$/\1/p; + s/^.*#\([[:digit:]]\+\))$/\1/p; s/^.*back[- ]*port[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip; s/^.*cherry[- ]*pick[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip") diff --git a/utils/simple-backport/format-changelog.py b/utils/simple-backport/format-changelog.py index ccda88c6809..5dff4f1c5e8 100755 --- a/utils/simple-backport/format-changelog.py +++ b/utils/simple-backport/format-changelog.py @@ -93,7 +93,7 @@ for line in args.file: # Normalize category name for c in categories_preferred_order: - if fuzzywuzzy.fuzz.ratio(pr['category'], c) >= 90: + if fuzzywuzzy.fuzz.ratio(pr['category'].lower(), c.lower()) >= 90: pr['category'] = c break From 8689797efc4d749f4cda139818b1a94caccfe628 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 10 Sep 2020 19:57:26 +0300 Subject: [PATCH 094/625] 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 095/625] 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 096/625] Bump CI [2]. From 5675efbd47fde50524463a14758c672091264897 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Sep 2020 20:16:12 +0300 Subject: [PATCH 097/625] Fix build --- src/Common/remapExecutable.cpp | 30 ++++++++++++++---------------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/src/Common/remapExecutable.cpp b/src/Common/remapExecutable.cpp index b41fece0c79..6b86e8717a4 100644 --- a/src/Common/remapExecutable.cpp +++ b/src/Common/remapExecutable.cpp @@ -89,20 +89,6 @@ __attribute__((__noinline__)) int64_t our_syscall(...) } -__attribute__((__always_inline__)) void our_memcpy(char * __restrict dst, const char * __restrict src, ssize_t n) -{ - while (n > 0) - { - _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), - _mm_loadu_si128(reinterpret_cast(src))); - - dst += 16; - src += 16; - n -= 16; - } -} - - __attribute__((__noinline__)) void remapToHugeStep3(void * scratch, size_t size, size_t offset) { /// The function should not use the stack, otherwise various optimizations, including "omit-frame-pointer" may break the code. @@ -122,7 +108,7 @@ __attribute__((__noinline__)) void remapToHugeStep2(void * begin, size_t size, v /** Unmap old memory region with the code of our program. * Our instruction pointer is located inside scratch area and this function can execute after old code is unmapped. * But it cannot call any other functions because they are not available at usual addresses - * - that's why we have to use "our_syscall" and "our_memcpy" functions. + * - that's why we have to use "our_syscall" function and a substitution for memcpy. * (Relative addressing may continue to work but we should not assume that). */ @@ -152,7 +138,19 @@ __attribute__((__noinline__)) void remapToHugeStep2(void * begin, size_t size, v /// Copy the code from scratch area to the old memory location. - our_memcpy(reinterpret_cast(begin), reinterpret_cast(scratch), size); + { + __m128i * __restrict dst = reinterpret_cast<__m128i *>(begin); + const __m128i * __restrict src = reinterpret_cast(scratch); + const __m128i * __restrict src_end = reinterpret_cast(reinterpret_cast(scratch) + size); + while (src < src_end) + { + _mm_storeu_si128(dst, _mm_loadu_si128(src)); + + ++dst; + ++src; + } + } + //syscall_func(SYS_write, 2, &dot, 1); /// Make the memory area with the code executable and non-writable. From fa6d88b3b29f9a0e852e009651c32652e0201fad Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 10 Sep 2020 21:43:02 +0300 Subject: [PATCH 098/625] 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 b3124d2cb0d8bae72f68d027d11cb40f2e6e4315 Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 10 Sep 2020 23:51:30 +0300 Subject: [PATCH 099/625] adding test outputs --- cmake_files_header.md | 12 +-- cmake_flags_and_output.md | 173 ++++++++++++++++++++++++++++++++++++ cmake_flags_md_generator.sh | 24 +++++ 3 files changed, 200 insertions(+), 9 deletions(-) create mode 100644 cmake_flags_and_output.md mode change 100644 => 100755 cmake_flags_md_generator.sh diff --git a/cmake_files_header.md b/cmake_files_header.md index bb613edaf1b..17f006a7760 100644 --- a/cmake_files_header.md +++ b/cmake_files_header.md @@ -22,12 +22,12 @@ If the option's purpose can't be guessed by its name, or the purpose guess may b the `option()` line and explain what it does. The best way would be linking the docs page (if it exists). The comment is parsed into a separate column (see below). -Even better: +Even better (default off value is omitted): ``` # Adds the ability to test ClickHouse using Google.Test (would produce another target unit_tests_dbms). # see tests/CMakeLists.txt for implementation detail. -option(ENABLE_GTEST_TESTS OFF) +option(ENABLE_GTEST_TESTS) ``` ### If the option's state could produce unwanted (or unusual) result, explicitly warn the user. @@ -113,7 +113,7 @@ Better: ``` # https://github.com/include-what-you-use/include-what-you-use -option (USE_INCLUDE_WHAT_YOU_USE OFF) +option (USE_INCLUDE_WHAT_YOU_USE) ``` ### Prefer consistent default values. @@ -129,9 +129,3 @@ Prefer the `ON/OFF` values, if possible. | Name | Description | Default value | Comment | |------|-------------|---------------|---------| -| \#0 | 89354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | -| \#1 | 90329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | -| \#2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | -| \#N | … | … | … | … | … | - - diff --git a/cmake_flags_and_output.md b/cmake_flags_and_output.md new file mode 100644 index 00000000000..8bfe00c5408 --- /dev/null +++ b/cmake_flags_and_output.md @@ -0,0 +1,173 @@ +# CMake flags + +## Developer's guide for adding new CMake options + +### Don't be obvious. Be informative. + +Bad: +``` +option (ENABLE_TESTS "Enables testing" OFF) +``` + +This description is quite useless as is neither gives the viewer any additional information nor explains the option +purpose. If the option's name is quite self-descriptive, prefer the empty description. + +Better: + +``` +option(ENABLE_TESTS OFF) +``` + +If the option's purpose can't be guessed by its name, or the purpose guess may be misleading, leave a comment above +the `option()` line and explain what it does. The best way would be linking the docs page (if it exists). +The comment is parsed into a separate column (see below). + +Even better (default off value is omitted): + +``` +# Adds the ability to test ClickHouse using Google.Test (would produce another target unit_tests_dbms). +# see tests/CMakeLists.txt for implementation detail. +option(ENABLE_GTEST_TESTS) +``` + +### If the option's state could produce unwanted (or unusual) result, explicitly warn the user. + +Suppose you have an option that may strip debug symbols from the ClickHouse's part. +This can speed up the linking process, but produces a binary that cannot be debugged. +In that case, prefer explicitly raising a warning telling the developer that he may be doing something wrong. +Also, such options should be disabled if applies. + +Bad: +``` +option(STRIP_DEBUG_SYMBOLS_FUNCTIONS + "Do not generate debugger info for ClickHouse functions. + ${STRIP_DSF_DEFAULT}) + +if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) + target_compile_options(clickhouse_functions PRIVATE "-g0") +endif() + +``` +Better: + +``` +# Provides faster linking and lower binary size. +# Tradeoff is the inability to debug some source files with e.g. gdb +# (empty stack frames and no local variables)." +option(STRIP_DEBUG_SYMBOLS_FUNCTIONS + "Do not generate debugger info for ClickHouse functions." + ${STRIP_DSF_DEFAULT}) + +if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) + message(WARNING "Not generating debugger info for ClickHouse functions") + target_compile_options(clickhouse_functions PRIVATE "-g0") +endif() +``` + +### In the option's description, explain WHAT the option does rather than WHY it does something. + +The WHY explanation should be placed in the comment. +You may find that the option's name is self-descriptive. + +Bad: + +``` +option(ENABLE_THINLTO "Enable Thin LTO. Only applicable for clang. It's also suppressed when building with tests or sanitizers." ON) +``` + +Better: + +``` +# Only applicable for clang. +# Turned off when building with tests or sanitizers. +option(ENABLE_THINLTO ON). +``` + +### Don't assume other developers know as much as you do. + +In ClickHouse, there are many tools used that an ordinary developer may not know. If you are in doubt, give a link to +the tool's docs. It won't take much of your time. + +Bad: + +``` +option(ENABLE_THINLTO "Enable Thin LTO. Only applicable for clang. It's also suppressed when building with tests or sanitizers." ON) +``` + +Better (combined with the above hint): + +``` +# https://clang.llvm.org/docs/ThinLTO.html +# Only applicable for clang. +# Turned off when building with tests or sanitizers. +option(ENABLE_THINLTO ON). +``` + +Other example, bad: + +``` +option (USE_INCLUDE_WHAT_YOU_USE "Use 'include-what-you-use' tool" OFF) +``` + +Better: + +``` +# https://github.com/include-what-you-use/include-what-you-use +option (USE_INCLUDE_WHAT_YOU_USE) +``` + +### Prefer consistent default values. + +CMake allows you to pass a plethora of values representing boolean `true/false`, e.g. `1, ON, YES, ...`. +Prefer the `ON/OFF` values, if possible. + + +## List of CMake flags + +* This list is auto-generated by [this bash script](bash.sh). +* The flag name is a link to its position in the code. + +| Name | Description | Default value | Comment | +|------|-------------|---------------|---------| +| ENABLE_CLANG_TIDY | OFF | Use 'clang-tidy' static analyzer if present | | + +| USE_INTERNAL_ | ${LIB_NAME_UC}_LIBRARY "Use bundled library ${LIB_NAME} instead of system" ${NOT_UNBUNDLED} | | | + +| FUZZER | | Enable fuzzer: libfuzzer | | + +| PARALLEL_COMPILE_JOBS | | Define the maximum number of concurrent compilation jobs" " | | + +| PARALLEL_LINK_JOBS | | Define the maximum number of concurrent link jobs" " | | + +| SANITIZE | | Enable sanitizer: address, memory, thread, undefined" " | | + +| LINKER_NAME | | Linker name or full path | | + +| WEVERYTHING | ON | Enables -Weverything option with some exceptions. This is intended for exploration of new compiler warnings that may be found to be useful. Only makes sense for clang. | | + +| ENABLE_CLICKHOUSE_ALL | ON | Enable all tools | | + +| ENABLE_CLICKHOUSE_SERVER | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-server | | + +| ENABLE_CLICKHOUSE_CLIENT | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-client | | + +| ENABLE_CLICKHOUSE_LOCAL | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-local | | + +| ENABLE_CLICKHOUSE_BENCHMARK | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-benchmark | | + +| ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-extract-from-config | | + +| ENABLE_CLICKHOUSE_COMPRESSOR | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-compressor | | + +| ENABLE_CLICKHOUSE_COPIER | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-copier | | + +| ENABLE_CLICKHOUSE_FORMAT | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-format | | + +| ENABLE_CLICKHOUSE_OBFUSCATOR | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-obfuscator | | + +| ENABLE_CLICKHOUSE_ODBC_BRIDGE | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-odbc-bridge | | + +| ENABLE_CLICKHOUSE_INSTALL | OFF | Enable clickhouse-install | | + +| ENABLE_CLICKHOUSE_INSTALL | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-install | | + diff --git a/cmake_flags_md_generator.sh b/cmake_flags_md_generator.sh old mode 100644 new mode 100755 index e72f7660e6d..390840039af --- a/cmake_flags_md_generator.sh +++ b/cmake_flags_md_generator.sh @@ -1 +1,25 @@ #!/bin/bash/ + +output_file_name="cmake_flags_and_output.md" + +regex='s/^((\s*#\s+.*\n?)*)\s*option\s*\(([A-Z_]+)\s*(\"((.|\n)*?)\")?\s*(.*)?\).*$/| \3 | \7 | \5 | \1 |\n/mg;t;d' + +rm -fr ${output_file_name} +touch ${output_file_name} +cat cmake_files_header.md >> ${output_file_name} + +process() { + for i in "$1"/*.cmake "$1"/CMakeLists.txt;do + if [ -d "$i" ];then + process "$i" + elif [ -f "$i" ]; then + echo "Processing $i" + + cat $i | sed -E "${regex}" >> ${output_file_name} + fi + done +} + +for base_folder in ./base ./cmake ./programs ./src; do + process $base_folder +done From 5dbfb3771be7d259175d04df7fac3b9e09974364 Mon Sep 17 00:00:00 2001 From: myrrc Date: Fri, 11 Sep 2020 00:22:02 +0300 Subject: [PATCH 100/625] updating links --- cmake_files_header.md | 4 +-- cmake_flags_and_output.md | 67 +++++++++++++------------------------ cmake_flags_md_generator.sh | 9 +++-- 3 files changed, 31 insertions(+), 49 deletions(-) diff --git a/cmake_files_header.md b/cmake_files_header.md index 17f006a7760..00a8b327e4c 100644 --- a/cmake_files_header.md +++ b/cmake_files_header.md @@ -127,5 +127,5 @@ Prefer the `ON/OFF` values, if possible. * This list is auto-generated by [this bash script](bash.sh). * The flag name is a link to its position in the code. -| Name | Description | Default value | Comment | -|------|-------------|---------------|---------| +| Name | Default value | Description | Comment | +|------|---------------|-------------|---------| diff --git a/cmake_flags_and_output.md b/cmake_flags_and_output.md index 8bfe00c5408..97b8049121f 100644 --- a/cmake_flags_and_output.md +++ b/cmake_flags_and_output.md @@ -127,47 +127,26 @@ Prefer the `ON/OFF` values, if possible. * This list is auto-generated by [this bash script](bash.sh). * The flag name is a link to its position in the code. -| Name | Description | Default value | Comment | -|------|-------------|---------------|---------| -| ENABLE_CLANG_TIDY | OFF | Use 'clang-tidy' static analyzer if present | | - -| USE_INTERNAL_ | ${LIB_NAME_UC}_LIBRARY "Use bundled library ${LIB_NAME} instead of system" ${NOT_UNBUNDLED} | | | - -| FUZZER | | Enable fuzzer: libfuzzer | | - -| PARALLEL_COMPILE_JOBS | | Define the maximum number of concurrent compilation jobs" " | | - -| PARALLEL_LINK_JOBS | | Define the maximum number of concurrent link jobs" " | | - -| SANITIZE | | Enable sanitizer: address, memory, thread, undefined" " | | - -| LINKER_NAME | | Linker name or full path | | - -| WEVERYTHING | ON | Enables -Weverything option with some exceptions. This is intended for exploration of new compiler warnings that may be found to be useful. Only makes sense for clang. | | - -| ENABLE_CLICKHOUSE_ALL | ON | Enable all tools | | - -| ENABLE_CLICKHOUSE_SERVER | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-server | | - -| ENABLE_CLICKHOUSE_CLIENT | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-client | | - -| ENABLE_CLICKHOUSE_LOCAL | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-local | | - -| ENABLE_CLICKHOUSE_BENCHMARK | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-benchmark | | - -| ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-extract-from-config | | - -| ENABLE_CLICKHOUSE_COMPRESSOR | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-compressor | | - -| ENABLE_CLICKHOUSE_COPIER | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-copier | | - -| ENABLE_CLICKHOUSE_FORMAT | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-format | | - -| ENABLE_CLICKHOUSE_OBFUSCATOR | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-obfuscator | | - -| ENABLE_CLICKHOUSE_ODBC_BRIDGE | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-odbc-bridge | | - -| ENABLE_CLICKHOUSE_INSTALL | OFF | Enable clickhouse-install | | - -| ENABLE_CLICKHOUSE_INSTALL | ${ENABLE_CLICKHOUSE_ALL} | Enable clickhouse-install | | - +| Name | Default value | Description | Comment | +|------|---------------|-------------|---------| +| (`ENABLE_CLANG_TIDY`)[http://github.com/clickhouse/clickhouse/blob/master/cmake/analysis.cmake] | `OFF` | Use 'clang-tidy' static analyzer if present | | +| (`USE_INTERNAL_`)[http://github.com/clickhouse/clickhouse/blob/master/cmake/contrib_finder.cmake] | `${LIB_NAME_UC}_LIBRARY "Use bundled library ${LIB_NAME} instead of system" ${NOT_UNBUNDLED}` | | | +| (`FUZZER`)[http://github.com/clickhouse/clickhouse/blob/master/cmake/fuzzer.cmake] | `` | Enable fuzzer: libfuzzer | | +| (`PARALLEL_COMPILE_JOBS`)[http://github.com/clickhouse/clickhouse/blob/master/cmake/limit_jobs.cmake] | `` | Define the maximum number of concurrent compilation jobs" " | | +| (`PARALLEL_LINK_JOBS`)[http://github.com/clickhouse/clickhouse/blob/master/cmake/limit_jobs.cmake] | `` | Define the maximum number of concurrent link jobs" " | | +| (`SANITIZE`)[http://github.com/clickhouse/clickhouse/blob/master/cmake/sanitize.cmake] | `` | Enable sanitizer: address, memory, thread, undefined" " | | +| (`LINKER_NAME`)[http://github.com/clickhouse/clickhouse/blob/master/cmake/tools.cmake] | `` | Linker name or full path | | +| (`WEVERYTHING`)[http://github.com/clickhouse/clickhouse/blob/master/cmake/warnings.cmake] | `ON` | Enables -Weverything option with some exceptions. This is intended for exploration of new compiler warnings that may be found to be useful. Only makes sense for clang. | | +| (`ENABLE_CLICKHOUSE_ALL`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `ON` | Enable all tools | | +| (`ENABLE_CLICKHOUSE_SERVER`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-server | | +| (`ENABLE_CLICKHOUSE_CLIENT`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-client | | +| (`ENABLE_CLICKHOUSE_LOCAL`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-local | | +| (`ENABLE_CLICKHOUSE_BENCHMARK`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-benchmark | | +| (`ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-extract-from-config | | +| (`ENABLE_CLICKHOUSE_COMPRESSOR`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-compressor | | +| (`ENABLE_CLICKHOUSE_COPIER`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-copier | | +| (`ENABLE_CLICKHOUSE_FORMAT`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-format | | +| (`ENABLE_CLICKHOUSE_OBFUSCATOR`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-obfuscator | | +| (`ENABLE_CLICKHOUSE_ODBC_BRIDGE`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-odbc-bridge | | +| (`ENABLE_CLICKHOUSE_INSTALL`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `OFF` | Enable clickhouse-install | | +| (`ENABLE_CLICKHOUSE_INSTALL`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-install | | diff --git a/cmake_flags_md_generator.sh b/cmake_flags_md_generator.sh index 390840039af..6796355222d 100755 --- a/cmake_flags_md_generator.sh +++ b/cmake_flags_md_generator.sh @@ -1,8 +1,7 @@ #!/bin/bash/ output_file_name="cmake_flags_and_output.md" - -regex='s/^((\s*#\s+.*\n?)*)\s*option\s*\(([A-Z_]+)\s*(\"((.|\n)*?)\")?\s*(.*)?\).*$/| \3 | \7 | \5 | \1 |\n/mg;t;d' +ch_master_url="http:\/\/github.com\/clickhouse\/clickhouse\/blob\/master\/" rm -fr ${output_file_name} touch ${output_file_name} @@ -14,8 +13,12 @@ process() { process "$i" elif [ -f "$i" ]; then echo "Processing $i" + subd_name=${i//\//\\/} + subd_name=${subd_name//\./\\\.} + subd_name=${subd_name:2} + regex='s/^((\s*#\s+.*\n?)*)\s*option\s*\(([A-Z_]+)\s*(\"((.|\n)*?)\")?\s*(.*)?\).*$/| (`\3`)['$ch_master_url${subd_name:2}'] | `\7` | \5 | \1 |/mg;t;d' - cat $i | sed -E "${regex}" >> ${output_file_name} + cat $i | sed -E "$regex" >> ${output_file_name} fi done } From 3319933dc9ecb463d65edcc724576192820d5b0a Mon Sep 17 00:00:00 2001 From: myrrc Date: Fri, 11 Sep 2020 00:48:56 +0300 Subject: [PATCH 101/625] fix: subsitution --- cmake_files_header.md | 22 +++--- cmake_flags_and_output.md | 151 ++++++++++++++++++++++++++++-------- cmake_flags_md_generator.sh | 28 ++++--- 3 files changed, 148 insertions(+), 53 deletions(-) diff --git a/cmake_files_header.md b/cmake_files_header.md index 00a8b327e4c..e1fa8ad29fe 100644 --- a/cmake_files_header.md +++ b/cmake_files_header.md @@ -5,7 +5,7 @@ ### Don't be obvious. Be informative. Bad: -``` +```cmake option (ENABLE_TESTS "Enables testing" OFF) ``` @@ -14,7 +14,7 @@ purpose. If the option's name is quite self-descriptive, prefer the empty descri Better: -``` +```cmake option(ENABLE_TESTS OFF) ``` @@ -24,7 +24,7 @@ The comment is parsed into a separate column (see below). Even better (default off value is omitted): -``` +```cmake # Adds the ability to test ClickHouse using Google.Test (would produce another target unit_tests_dbms). # see tests/CMakeLists.txt for implementation detail. option(ENABLE_GTEST_TESTS) @@ -38,7 +38,7 @@ In that case, prefer explicitly raising a warning telling the developer that he Also, such options should be disabled if applies. Bad: -``` +```cmake option(STRIP_DEBUG_SYMBOLS_FUNCTIONS "Do not generate debugger info for ClickHouse functions. ${STRIP_DSF_DEFAULT}) @@ -50,7 +50,7 @@ endif() ``` Better: -``` +```cmake # Provides faster linking and lower binary size. # Tradeoff is the inability to debug some source files with e.g. gdb # (empty stack frames and no local variables)." @@ -71,13 +71,13 @@ You may find that the option's name is self-descriptive. Bad: -``` +```cmake option(ENABLE_THINLTO "Enable Thin LTO. Only applicable for clang. It's also suppressed when building with tests or sanitizers." ON) ``` Better: -``` +```cmake # Only applicable for clang. # Turned off when building with tests or sanitizers. option(ENABLE_THINLTO ON). @@ -90,13 +90,13 @@ the tool's docs. It won't take much of your time. Bad: -``` +```cmake option(ENABLE_THINLTO "Enable Thin LTO. Only applicable for clang. It's also suppressed when building with tests or sanitizers." ON) ``` Better (combined with the above hint): -``` +```cmake # https://clang.llvm.org/docs/ThinLTO.html # Only applicable for clang. # Turned off when building with tests or sanitizers. @@ -105,13 +105,13 @@ option(ENABLE_THINLTO ON). Other example, bad: -``` +```cmake option (USE_INCLUDE_WHAT_YOU_USE "Use 'include-what-you-use' tool" OFF) ``` Better: -``` +```cmake # https://github.com/include-what-you-use/include-what-you-use option (USE_INCLUDE_WHAT_YOU_USE) ``` diff --git a/cmake_flags_and_output.md b/cmake_flags_and_output.md index 97b8049121f..65ad40dc916 100644 --- a/cmake_flags_and_output.md +++ b/cmake_flags_and_output.md @@ -5,7 +5,7 @@ ### Don't be obvious. Be informative. Bad: -``` +```cmake option (ENABLE_TESTS "Enables testing" OFF) ``` @@ -14,7 +14,7 @@ purpose. If the option's name is quite self-descriptive, prefer the empty descri Better: -``` +```cmake option(ENABLE_TESTS OFF) ``` @@ -24,7 +24,7 @@ The comment is parsed into a separate column (see below). Even better (default off value is omitted): -``` +```cmake # Adds the ability to test ClickHouse using Google.Test (would produce another target unit_tests_dbms). # see tests/CMakeLists.txt for implementation detail. option(ENABLE_GTEST_TESTS) @@ -38,7 +38,7 @@ In that case, prefer explicitly raising a warning telling the developer that he Also, such options should be disabled if applies. Bad: -``` +```cmake option(STRIP_DEBUG_SYMBOLS_FUNCTIONS "Do not generate debugger info for ClickHouse functions. ${STRIP_DSF_DEFAULT}) @@ -50,7 +50,7 @@ endif() ``` Better: -``` +```cmake # Provides faster linking and lower binary size. # Tradeoff is the inability to debug some source files with e.g. gdb # (empty stack frames and no local variables)." @@ -71,13 +71,13 @@ You may find that the option's name is self-descriptive. Bad: -``` +```cmake option(ENABLE_THINLTO "Enable Thin LTO. Only applicable for clang. It's also suppressed when building with tests or sanitizers." ON) ``` Better: -``` +```cmake # Only applicable for clang. # Turned off when building with tests or sanitizers. option(ENABLE_THINLTO ON). @@ -90,13 +90,13 @@ the tool's docs. It won't take much of your time. Bad: -``` +```cmake option(ENABLE_THINLTO "Enable Thin LTO. Only applicable for clang. It's also suppressed when building with tests or sanitizers." ON) ``` Better (combined with the above hint): -``` +```cmake # https://clang.llvm.org/docs/ThinLTO.html # Only applicable for clang. # Turned off when building with tests or sanitizers. @@ -105,13 +105,13 @@ option(ENABLE_THINLTO ON). Other example, bad: -``` +```cmake option (USE_INCLUDE_WHAT_YOU_USE "Use 'include-what-you-use' tool" OFF) ``` Better: -``` +```cmake # https://github.com/include-what-you-use/include-what-you-use option (USE_INCLUDE_WHAT_YOU_USE) ``` @@ -124,29 +124,114 @@ Prefer the `ON/OFF` values, if possible. ## List of CMake flags -* This list is auto-generated by [this bash script](bash.sh). +* This list is auto-generated by [this bash script](cmake_flags_md_generator.sh). * The flag name is a link to its position in the code. | Name | Default value | Description | Comment | |------|---------------|-------------|---------| -| (`ENABLE_CLANG_TIDY`)[http://github.com/clickhouse/clickhouse/blob/master/cmake/analysis.cmake] | `OFF` | Use 'clang-tidy' static analyzer if present | | -| (`USE_INTERNAL_`)[http://github.com/clickhouse/clickhouse/blob/master/cmake/contrib_finder.cmake] | `${LIB_NAME_UC}_LIBRARY "Use bundled library ${LIB_NAME} instead of system" ${NOT_UNBUNDLED}` | | | -| (`FUZZER`)[http://github.com/clickhouse/clickhouse/blob/master/cmake/fuzzer.cmake] | `` | Enable fuzzer: libfuzzer | | -| (`PARALLEL_COMPILE_JOBS`)[http://github.com/clickhouse/clickhouse/blob/master/cmake/limit_jobs.cmake] | `` | Define the maximum number of concurrent compilation jobs" " | | -| (`PARALLEL_LINK_JOBS`)[http://github.com/clickhouse/clickhouse/blob/master/cmake/limit_jobs.cmake] | `` | Define the maximum number of concurrent link jobs" " | | -| (`SANITIZE`)[http://github.com/clickhouse/clickhouse/blob/master/cmake/sanitize.cmake] | `` | Enable sanitizer: address, memory, thread, undefined" " | | -| (`LINKER_NAME`)[http://github.com/clickhouse/clickhouse/blob/master/cmake/tools.cmake] | `` | Linker name or full path | | -| (`WEVERYTHING`)[http://github.com/clickhouse/clickhouse/blob/master/cmake/warnings.cmake] | `ON` | Enables -Weverything option with some exceptions. This is intended for exploration of new compiler warnings that may be found to be useful. Only makes sense for clang. | | -| (`ENABLE_CLICKHOUSE_ALL`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `ON` | Enable all tools | | -| (`ENABLE_CLICKHOUSE_SERVER`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-server | | -| (`ENABLE_CLICKHOUSE_CLIENT`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-client | | -| (`ENABLE_CLICKHOUSE_LOCAL`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-local | | -| (`ENABLE_CLICKHOUSE_BENCHMARK`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-benchmark | | -| (`ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-extract-from-config | | -| (`ENABLE_CLICKHOUSE_COMPRESSOR`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-compressor | | -| (`ENABLE_CLICKHOUSE_COPIER`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-copier | | -| (`ENABLE_CLICKHOUSE_FORMAT`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-format | | -| (`ENABLE_CLICKHOUSE_OBFUSCATOR`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-obfuscator | | -| (`ENABLE_CLICKHOUSE_ODBC_BRIDGE`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-odbc-bridge | | -| (`ENABLE_CLICKHOUSE_INSTALL`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `OFF` | Enable clickhouse-install | | -| (`ENABLE_CLICKHOUSE_INSTALL`)[http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt] | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-install | | +| [`ENABLE_IPO`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `` | Enable full link time optimization | | +| [`USE_STATIC_LIBRARIES`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `ON` | Set to FALSE to use shared libraries | | +| [`MAKE_STATIC_LIBRARIES`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `${USE_STATIC_LIBRARIES}` | Set to FALSE to make shared libraries | | +| [`SPLIT_SHARED_LIBRARIES`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `OFF` | DEV ONLY. Keep all internal libs as separate .so for faster linking | | +| [`CLICKHOUSE_SPLIT_BINARY`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `OFF` | Make several binaries instead one bundled (clickhouse-server, clickhouse-client, ... ) | | +| [`ENABLE_FUZZING`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `OFF` | Enables fuzzing instrumentation | | +| [`ENABLE_TESTS`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `ON` | Enables tests | | +| [`GLIBC_COMPATIBILITY`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `ON` | Set to TRUE to enable compatibility with older glibc libraries. Only for x86_64, Linux. Implies ENABLE_FASTMEMCPY. | | +| [`ADD_GDB_INDEX_FOR_GOLD`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `0` | Set to add .gdb-index to resulting binaries for gold linker. NOOP if lld is used. | | +| [`COMPILER_PIPE`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `ON` | -pipe compiler option [less /tmp usage, more ram usage] | | +| [`ARCH_NATIVE`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `0` | Enable -march=native compiler flag | | +| [`WITH_COVERAGE`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `0` | Build with coverage. | | +| [`ENABLE_THINLTO`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `ON` | Enable Thin LTO. Only applicable for clang. It's also suppressed when building with tests or sanitizers. | | +| [`ENABLE_LIBRARIES`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `ON` | Enable all libraries (Global default switch) | | +| [`UNBUNDLED`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `OFF` | Try find all libraries in system. We recommend to avoid this mode for production builds, because we cannot guarantee exact versions and variants of libraries your system has installed. This mode exists for enthusiastic developers who search for trouble. Also it is useful for maintainers of OS packages. | | +| [`WERROR`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `OFF` | Enable -Werror compiler option | | +| [`WERROR`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `ON` | Enable -Werror compiler option | | +| [`USE_INCLUDE_WHAT_YOU_USE`](http://github.com/clickhouse/clickhouse/blob/master//CMakeLists.txt) | `OFF` | Use 'include-what-you-use' tool | | +| [`ENABLE_CLANG_TIDY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/analysis.cmake) | `OFF` | Use 'clang-tidy' static analyzer if present | | +| [`USE_INTERNAL_`](http://github.com/clickhouse/clickhouse/blob/master/cmake/contrib_finder.cmake) | `${LIB_NAME_UC}_LIBRARY "Use bundled library ${LIB_NAME} instead of system" ${NOT_UNBUNDLED}` | | | +| [`FUZZER`](http://github.com/clickhouse/clickhouse/blob/master/cmake/fuzzer.cmake) | `` | Enable fuzzer: libfuzzer | | +| [`PARALLEL_COMPILE_JOBS`](http://github.com/clickhouse/clickhouse/blob/master/cmake/limit_jobs.cmake) | `` | Define the maximum number of concurrent compilation jobs" " | | +| [`PARALLEL_LINK_JOBS`](http://github.com/clickhouse/clickhouse/blob/master/cmake/limit_jobs.cmake) | `` | Define the maximum number of concurrent link jobs" " | | +| [`SANITIZE`](http://github.com/clickhouse/clickhouse/blob/master/cmake/sanitize.cmake) | `` | Enable sanitizer: address, memory, thread, undefined" " | | +| [`LINKER_NAME`](http://github.com/clickhouse/clickhouse/blob/master/cmake/tools.cmake) | `` | Linker name or full path | | +| [`WEVERYTHING`](http://github.com/clickhouse/clickhouse/blob/master/cmake/warnings.cmake) | `ON` | Enables -Weverything option with some exceptions. This is intended for exploration of new compiler warnings that may be found to be useful. Only makes sense for clang. | | +| [`ENABLE_AMQPCPP`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/amqpcpp.cmake) | `${ENABLE_LIBRARIES}` | Enalbe AMQP-CPP | | +| [`ENABLE_AVRO`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/avro.cmake) | `${ENABLE_LIBRARIES}` | Enable Avro | | +| [`ENABLE_BASE`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/base64.cmake) | `64 "Enable base64" ${ENABLE_LIBRARIES}` | | | +| [`ENABLE_BROTLI`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/brotli.cmake) | `${ENABLE_LIBRARIES}` | Enable brotli | | +| [`USE_INTERNAL_BROTLI_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/brotli.cmake) | `${USE_STATIC_LIBRARIES}` | Set to FALSE to use system libbrotli library instead of bundled | | +| [`USE_INTERNAL_BROTLI_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/brotli.cmake) | `ON` | Set to FALSE to use system libbrotli library instead of bundled | | +| [`ENABLE_CAPNP`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/capnp.cmake) | `${ENABLE_LIBRARIES}` | Enable Cap'n Proto | | +| [`USE_INTERNAL_CAPNP_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/capnp.cmake) | `${NOT_UNBUNDLED}` | Set to FALSE to use system capnproto library instead of bundled | | +| [`ENABLE_CASSANDRA`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/cassandra.cmake) | `${ENABLE_LIBRARIES}` | Enable Cassandra | | +| [`ENABLE_CCACHE`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/ccache.cmake) | `${ENABLE_CCACHE_BY_DEFAULT}` | Speedup re-compilations using ccache | | +| [`ENABLE_CURL`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/curl.cmake) | `${ENABLE_LIBRARIES}` | Enable curl | | +| [`USE_INTERNAL_CURL`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/curl.cmake) | `${NOT_UNBUNDLED}` | Use internal curl library | | +| [`USE_LIBCXX`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/cxx.cmake) | `${NOT_UNBUNDLED}` | Use libc++ and libc++abi instead of libstdc++ | | +| [`USE_INTERNAL_LIBCXX_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/cxx.cmake) | `${USE_INTERNAL_LIBCXX_LIBRARY_DEFAULT}` | Set to FALSE to use system libcxx and libcxxabi libraries instead of bundled | | +| [`ENABLE_FASTOPS`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/fastops.cmake) | `${ENABLE_LIBRARIES}` | Enable fast vectorized mathematical functions library by Mikhail Parakhin | | +| [`ENABLE_GPERF`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/gperf.cmake) | `${ENABLE_LIBRARIES}` | Use gperf function hash generator tool | | +| [`ENABLE_GRPC`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/grpc.cmake) | `${ENABLE_LIBRARIES}` | Use gRPC | | +| [`ENABLE_GTEST_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/gtest.cmake) | `${ENABLE_LIBRARIES}` | Enable gtest library | | +| [`USE_INTERNAL_GTEST_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/gtest.cmake) | `${NOT_UNBUNDLED}` | Set to FALSE to use system Google Test instead of bundled | | +| [`ENABLE_H`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/h3.cmake) | `3 "Enable H3" ${ENABLE_LIBRARIES}` | | | +| [`ENABLE_HDFS`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/hdfs3.cmake) | `${ENABLE_LIBRARIES}` | Enable HDFS | | +| [`USE_INTERNAL_HDFS`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/hdfs3.cmake) | `3_LIBRARY "Set to FALSE to use system HDFS3 instead of bundled (experimental - set to OFF on your own risk` | | | +| [`ENABLE_ICU`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/icu.cmake) | `${ENABLE_LIBRARIES}` | Enable ICU | | +| [`ENABLE_ICU`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/icu.cmake) | `0` | Enable ICU | | +| [`USE_INTERNAL_ICU_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/icu.cmake) | `${NOT_UNBUNDLED}` | Set to FALSE to use system ICU library instead of bundled | | +| [`ENABLE_LDAP`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/ldap.cmake) | `${ENABLE_LIBRARIES}` | Enable LDAP | | +| [`USE_INTERNAL_LDAP_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/ldap.cmake) | `${NOT_UNBUNDLED}` | Set to FALSE to use system *LDAP library instead of bundled | | +| [`ENABLE_GSASL_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/libgsasl.cmake) | `${ENABLE_LIBRARIES}` | Enable gsasl library | | +| [`USE_INTERNAL_LIBGSASL_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/libgsasl.cmake) | `${USE_STATIC_LIBRARIES}` | Set to FALSE to use system libgsasl library instead of bundled | | +| [`USE_INTERNAL_LIBGSASL_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/libgsasl.cmake) | `ON` | Set to FALSE to use system libgsasl library instead of bundled | | +| [`USE_INTERNAL_LIBXML`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/libxml2.cmake) | `2_LIBRARY "Set to FALSE to use system libxml2 library instead of bundled" ${NOT_UNBUNDLED}` | | | +| [`ENABLE_EMBEDDED_COMPILER`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/llvm.cmake) | `${ENABLE_LIBRARIES}` | Set to TRUE to enable support for 'compile_expressions' option for query execution | | +| [`USE_INTERNAL_LLVM_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/llvm.cmake) | `${NOT_UNBUNDLED}` | Use bundled or system LLVM library. | | +| [`LLVM_HAS_RTTI`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/llvm.cmake) | `ON` | Enable if LLVM was build with RTTI enabled | | +| [`ENABLE_MSGPACK`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/msgpack.cmake) | `${ENABLE_LIBRARIES}` | Enable msgpack library | | +| [`USE_INTERNAL_MSGPACK_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/msgpack.cmake) | `${NOT_UNBUNDLED}` | Set to FALSE to use system msgpack library instead of bundled | | +| [`ENABLE_MYSQL`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/mysqlclient.cmake) | `${ENABLE_LIBRARIES}` | Enable MySQL | | +| [`ENABLE_MYSQL`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/mysqlclient.cmake) | `FALSE` | Enable MySQL | | +| [`USE_INTERNAL_MYSQL_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/mysqlclient.cmake) | `${NOT_UNBUNDLED}` | Set to FALSE to use system mysqlclient library instead of bundled | | +| [`ENABLE_ODBC`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/odbc.cmake) | `${ENABLE_LIBRARIES}` | Enable ODBC library | | +| [`USE_INTERNAL_ODBC_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/odbc.cmake) | `${NOT_UNBUNDLED}` | Use internal ODBC library | | +| [`ENABLE_OPENCL`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/opencl.cmake) | `${ENABLE_LIBRARIES}` | Enable OpenCL support | | +| [`ENABLE_ORC`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/orc.cmake) | `${ENABLE_LIBRARIES}` | Enable ORC | | +| [`USE_INTERNAL_ORC_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/orc.cmake) | `"Set to FALSE to use system ORC instead of bundled (experimental set to OFF on your own risk` | | | +| [`ENABLE_PARQUET`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/parquet.cmake) | `${ENABLE_LIBRARIES}` | Enable parquet | | +| [`USE_INTERNAL_PARQUET_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/parquet.cmake) | `${NOT_UNBUNDLED}` | Set to FALSE to use system parquet library instead of bundled | | +| [`USE_INTERNAL_POCO_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/poco.cmake) | `ON` | Use internal Poco library | | +| [`ENABLE_PROTOBUF`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/protobuf.cmake) | `${ENABLE_LIBRARIES}` | Enable protobuf | | +| [`USE_INTERNAL_PROTOBUF_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/protobuf.cmake) | `${NOT_UNBUNDLED}` | Set to FALSE to use system protobuf instead of bundled | | +| [`ENABLE_RAPIDJSON`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/rapidjson.cmake) | `${ENABLE_LIBRARIES}` | Use rapidjson | | +| [`USE_INTERNAL_RAPIDJSON_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/rapidjson.cmake) | `${NOT_UNBUNDLED}` | Set to FALSE to use system rapidjson library instead of bundled | | +| [`ENABLE_RDKAFKA`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/rdkafka.cmake) | `${ENABLE_LIBRARIES}` | Enable kafka | | +| [`USE_INTERNAL_RDKAFKA_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/rdkafka.cmake) | `${NOT_UNBUNDLED}` | Set to FALSE to use system librdkafka instead of the bundled | | +| [`USE_INTERNAL_RE`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/re2.cmake) | `2_LIBRARY "Set to FALSE to use system re2 library instead of bundled [slower]" ${NOT_UNBUNDLED}` | | | +| [`ENABLE_S`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/s3.cmake) | `3 "Enable S3" ${ENABLE_LIBRARIES}` | | | +| [`USE_INTERNAL_AWS_S`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/s3.cmake) | `3_LIBRARY "Set to FALSE to use system S3 instead of bundled (experimental set to OFF on your own risk` | | | +| [`USE_SENTRY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/sentry.cmake) | `${ENABLE_LIBRARIES}` | Use Sentry | | +| [`USE_SIMDJSON`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/simdjson.cmake) | `${ENABLE_LIBRARIES}` | Use simdjson | | +| [`USE_SNAPPY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/snappy.cmake) | `${ENABLE_LIBRARIES}` | Enable support of snappy library | | +| [`USE_INTERNAL_SNAPPY_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/snappy.cmake) | `${NOT_UNBUNDLED}` | Set to FALSE to use system snappy library instead of bundled | | +| [`ENABLE_SSL`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/ssl.cmake) | `${ENABLE_LIBRARIES}` | Enable ssl | | +| [`USE_INTERNAL_SSL_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/ssl.cmake) | `${NOT_UNBUNDLED}` | Set to FALSE to use system *ssl library instead of bundled | | +| [`ENABLE_STATS`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/stats.cmake) | `${ENABLE_LIBRARIES}` | Enalbe StatsLib library | | +| [`USE_UNWIND`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/unwind.cmake) | `${ENABLE_LIBRARIES}` | Enable libunwind (better stacktraces) | | +| [`USE_INTERNAL_ZLIB_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/zlib.cmake) | `${NOT_UNBUNDLED}` | Set to FALSE to use system zlib library instead of bundled | | +| [`USE_INTERNAL_ZSTD_LIBRARY`](http://github.com/clickhouse/clickhouse/blob/master/cmake/find/zstd.cmake) | `${NOT_UNBUNDLED}` | Set to FALSE to use system zstd library instead of bundled | | +| [`ENABLE_CLICKHOUSE_ALL`](http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt) | `ON` | Enable all tools | | +| [`ENABLE_CLICKHOUSE_SERVER`](http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt) | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-server | | +| [`ENABLE_CLICKHOUSE_CLIENT`](http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt) | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-client | | +| [`ENABLE_CLICKHOUSE_LOCAL`](http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt) | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-local | | +| [`ENABLE_CLICKHOUSE_BENCHMARK`](http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt) | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-benchmark | | +| [`ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG`](http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt) | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-extract-from-config | | +| [`ENABLE_CLICKHOUSE_COMPRESSOR`](http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt) | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-compressor | | +| [`ENABLE_CLICKHOUSE_COPIER`](http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt) | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-copier | | +| [`ENABLE_CLICKHOUSE_FORMAT`](http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt) | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-format | | +| [`ENABLE_CLICKHOUSE_OBFUSCATOR`](http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt) | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-obfuscator | | +| [`ENABLE_CLICKHOUSE_ODBC_BRIDGE`](http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt) | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-odbc-bridge | | +| [`ENABLE_CLICKHOUSE_INSTALL`](http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt) | `OFF` | Enable clickhouse-install | | +| [`ENABLE_CLICKHOUSE_INSTALL`](http://github.com/clickhouse/clickhouse/blob/master/programs/CMakeLists.txt) | `${ENABLE_CLICKHOUSE_ALL}` | Enable clickhouse-install | | +| [`ENABLE_MULTITARGET_CODE`](http://github.com/clickhouse/clickhouse/blob/master/src/Functions/CMakeLists.txt) | `ON` | | | diff --git a/cmake_flags_md_generator.sh b/cmake_flags_md_generator.sh index 6796355222d..17ccc8a1bba 100755 --- a/cmake_flags_md_generator.sh +++ b/cmake_flags_md_generator.sh @@ -1,5 +1,7 @@ #!/bin/bash/ +# https://regex101.com/r/R6iogw/7 + output_file_name="cmake_flags_and_output.md" ch_master_url="http:\/\/github.com\/clickhouse\/clickhouse\/blob\/master\/" @@ -9,20 +11,28 @@ cat cmake_files_header.md >> ${output_file_name} process() { for i in "$1"/*.cmake "$1"/CMakeLists.txt;do - if [ -d "$i" ];then - process "$i" - elif [ -f "$i" ]; then - echo "Processing $i" - subd_name=${i//\//\\/} - subd_name=${subd_name//\./\\\.} - subd_name=${subd_name:2} - regex='s/^((\s*#\s+.*\n?)*)\s*option\s*\(([A-Z_]+)\s*(\"((.|\n)*?)\")?\s*(.*)?\).*$/| (`\3`)['$ch_master_url${subd_name:2}'] | `\7` | \5 | \1 |/mg;t;d' + echo "Processing $i" + subd_name=${i//\//\\/} + subd_name=${subd_name//\./\\\.} + subd_name=${subd_name:2} + regex='s/^((\s*#\s+.*\n?)*)\s*option\s*\(([A-Z_]+)\s*(\"((.|\n)*?)\")?\s*(.*)?\).*$/| [`\3`]('$ch_master_url${subd_name:2}') | `\7` | \5 | \1 |/mg;t;d' + if [ -f $i ]; then cat $i | sed -E "$regex" >> ${output_file_name} fi done + + if [ "$2" = true ] ; then + for d in "$1"/*;do + if [ -d "$d" ];then + process $d + fi + done + fi } +process ./ false + for base_folder in ./base ./cmake ./programs ./src; do - process $base_folder + process $base_folder true done From f2a5216e97f1283b373720717f1d6f7ac79af86d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 11 Sep 2020 02:24:16 +0300 Subject: [PATCH 102/625] add waiting for fsync in WAL --- src/Common/FileSyncGuard.h | 2 +- src/Storages/MergeTree/MergeTreeSettings.h | 1 + src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp | 14 +++++++++----- src/Storages/MergeTree/MergeTreeWriteAheadLog.h | 5 +++-- utils/durability-test/durability-test.sh | 12 ++++++++++++ 5 files changed, 26 insertions(+), 8 deletions(-) diff --git a/src/Common/FileSyncGuard.h b/src/Common/FileSyncGuard.h index 5ec9b1d0c98..6451f6ebf36 100644 --- a/src/Common/FileSyncGuard.h +++ b/src/Common/FileSyncGuard.h @@ -17,7 +17,7 @@ public: FileSyncGuard(const DiskPtr & disk_, int fd_) : disk(disk_), fd(fd_) {} FileSyncGuard(const DiskPtr & disk_, const String & path) - : disk(disk_), fd(disk_->open(path, O_RDONLY)) {} + : disk(disk_), fd(disk_->open(path, O_RDWR)) {} ~FileSyncGuard() { diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 3f8f44dc11e..8652a6ef691 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -47,6 +47,7 @@ struct Settings; M(Bool, fsync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ M(UInt64, write_ahead_log_bytes_to_fsync, 100ULL * 1024 * 1024, "Amount of bytes, accumulated in WAL to do fsync.", 0) \ M(UInt64, write_ahead_log_interval_ms_to_fsync, 100, "Interval in milliseconds after which fsync for WAL is being done.", 0) \ + M(Bool, in_memory_parts_insert_sync, false, "If true insert of part with in-memory format will wait for fsync of WAL", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 5cfe9017248..bc6738a8321 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -33,6 +33,7 @@ MergeTreeWriteAheadLog::MergeTreeWriteAheadLog( std::lock_guard lock(write_mutex); out->sync(); sync_scheduled = false; + sync_cv.notify_all(); }); } @@ -50,7 +51,7 @@ void MergeTreeWriteAheadLog::init() void MergeTreeWriteAheadLog::addPart(const Block & block, const String & part_name) { - std::lock_guard lock(write_mutex); + std::unique_lock lock(write_mutex); auto part_info = MergeTreePartInfo::fromPartName(part_name, storage.format_version); min_block_number = std::min(min_block_number, part_info.min_block); @@ -70,7 +71,7 @@ void MergeTreeWriteAheadLog::addPart(const Block & block, const String & part_na void MergeTreeWriteAheadLog::dropPart(const String & part_name) { - std::lock_guard lock(write_mutex); + std::unique_lock lock(write_mutex); writeIntBinary(static_cast(0), *out); writeIntBinary(static_cast(ActionType::DROP_PART), *out); @@ -78,7 +79,7 @@ void MergeTreeWriteAheadLog::dropPart(const String & part_name) sync(lock); } -void MergeTreeWriteAheadLog::rotate(const std::lock_guard &) +void MergeTreeWriteAheadLog::rotate(const std::unique_lock &) { String new_name = String(WAL_FILE_NAME) + "_" + toString(min_block_number) + "_" @@ -90,7 +91,7 @@ void MergeTreeWriteAheadLog::rotate(const std::lock_guard &) MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const StorageMetadataPtr & metadata_snapshot) { - std::lock_guard lock(write_mutex); + std::unique_lock lock(write_mutex); MergeTreeData::MutableDataPartsVector parts; auto in = disk->readFile(path, DBMS_DEFAULT_BUFFER_SIZE); @@ -185,7 +186,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor return result; } -void MergeTreeWriteAheadLog::sync(const std::lock_guard &) +void MergeTreeWriteAheadLog::sync(std::unique_lock & lock) { size_t bytes_to_sync = storage.getSettings()->write_ahead_log_bytes_to_fsync; time_t time_to_sync = storage.getSettings()->write_ahead_log_interval_ms_to_fsync; @@ -201,6 +202,9 @@ void MergeTreeWriteAheadLog::sync(const std::lock_guard &) sync_task->scheduleAfter(time_to_sync); sync_scheduled = true; } + + if (storage.getSettings()->in_memory_parts_insert_sync) + sync_cv.wait(lock, [this] { return !sync_scheduled; }); } std::optional diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index 43abf3c04be..c5675eac696 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -44,8 +44,8 @@ public: private: void init(); - void rotate(const std::lock_guard & lock); - void sync(const std::lock_guard & lock); + void rotate(const std::unique_lock & lock); + void sync(std::unique_lock & lock); const MergeTreeData & storage; DiskPtr disk; @@ -60,6 +60,7 @@ private: BackgroundSchedulePool & pool; BackgroundSchedulePoolTaskHolder sync_task; + std::condition_variable sync_cv; size_t bytes_at_last_sync = 0; bool sync_scheduled = false; diff --git a/utils/durability-test/durability-test.sh b/utils/durability-test/durability-test.sh index c7f8936ec95..97c39473b69 100755 --- a/utils/durability-test/durability-test.sh +++ b/utils/durability-test/durability-test.sh @@ -1,5 +1,17 @@ #!/bin/bash +: ' +A simple test for durability. It starts up clickhouse server in qemu VM and runs +inserts via clickhouse benchmark tool. Then it kills VM in random moment and +checks whether table contains broken parts. With enabled fsync no broken parts +should be appeared. + +Usage: + +./install.sh +./durability-test.sh
+' + URL=http://cloud-images.ubuntu.com/bionic/current IMAGE=bionic-server-cloudimg-amd64.img SSH_PORT=11022 From 8a201a28c04e06f1a2ebb03d51c0e1d8e983680a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 11 Sep 2020 03:14:35 +0300 Subject: [PATCH 103/625] 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 104/625] 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 105/625] 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 3b9ab3f1be330b5ae7ffd7c68fd629ad3ebc9f6b Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 11 Sep 2020 12:23:31 +0300 Subject: [PATCH 106/625] Fix if --- src/Functions/if.cpp | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 20848bede32..584bed3f8c5 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -604,7 +604,6 @@ private: const ColumnUInt8 * cond_col, Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) { /// Convert both columns to the common type (if needed). - const ColumnWithTypeAndName & arg1 = block.getByPosition(arguments[1]); const ColumnWithTypeAndName & arg2 = block.getByPosition(arguments[2]); @@ -765,10 +764,22 @@ private: return ColumnNullable::create(materialized, ColumnUInt8::create(column->size(), 0)); } - static ColumnPtr getNestedColumn(const ColumnPtr & column) + /// Return nested column recursively removing Nullable, examples: + /// Nullable(size = 1, Int32(size = 1), UInt8(size = 1)) -> Int32(size = 1) + /// Const(size = 0, Nullable(size = 1, Int32(size = 1), UInt8(size = 1))) -> + /// Const(size = 0, Int32(size = 1)) + static ColumnPtr recursiveGetNestedColumnWithoutNullable(const ColumnPtr & column) { if (const auto * nullable = checkAndGetColumn(*column)) + { + /// Nullable cannot contain Nullable return nullable->getNestedColumnPtr(); + } + else if (const auto * column_const = checkAndGetColumn(*column)) + { + /// Save Constant, but remove Nullable + return ColumnConst::create(recursiveGetNestedColumnWithoutNullable(column_const->getDataColumnPtr()), column->size()); + } return column; } @@ -826,12 +837,12 @@ private: { arg_cond, { - getNestedColumn(arg_then.column), + recursiveGetNestedColumnWithoutNullable(arg_then.column), removeNullable(arg_then.type), "" }, { - getNestedColumn(arg_else.column), + recursiveGetNestedColumnWithoutNullable(arg_else.column), removeNullable(arg_else.type), "" }, From e25b1da29fa168b24464c83c1f661b363916afad Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 11 Sep 2020 13:53:26 +0300 Subject: [PATCH 107/625] Disable -Wstringop-overflow for gcc-10 --- cmake/warnings.cmake | 11 +++++++++-- src/Storages/MergeTree/MergeTreePartition.cpp | 8 -------- 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index aec3e46ffa6..6b26b9b95a5 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -169,9 +169,16 @@ elseif (COMPILER_GCC) # Warn if vector operation is not implemented via SIMD capabilities of the architecture add_cxx_compile_options(-Wvector-operation-performance) - # XXX: gcc10 stuck with this option while compiling GatherUtils code - # (anyway there are builds with clang, that will warn) if (CMAKE_CXX_COMPILER_VERSION VERSION_GREATER_EQUAL 10) + # XXX: gcc10 stuck with this option while compiling GatherUtils code + # (anyway there are builds with clang, that will warn) add_cxx_compile_options(-Wno-sequence-point) + # XXX: gcc10 false positive with this warning in MergeTreePartition.cpp + # inlined from 'void writeHexByteLowercase(UInt8, void*)' at ../src/Common/hex.h:39:11, + # inlined from 'DB::String DB::MergeTreePartition::getID(const DB::Block&) const' at ../src/Storages/MergeTree/MergeTreePartition.cpp:85:30: + # ../contrib/libc-headers/x86_64-linux-gnu/bits/string_fortified.h:34:33: error: writing 2 bytes into a region of size 0 [-Werror=stringop-overflow=] + # 34 | return __builtin___memcpy_chk (__dest, __src, __len, __bos0 (__dest)); + # For some reason (bug in gcc?) macro 'GCC diagnostic ignored "-Wstringop-overflow"' doesn't help. + add_cxx_compile_options(-Wno-stringop-overflow) endif() endif () diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 2802b842f54..880a3aa181d 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -29,11 +29,6 @@ String MergeTreePartition::getID(const MergeTreeData & storage) const return getID(storage.getInMemoryMetadataPtr()->getPartitionKey().sample_block); } -#if defined (__GNUC__) && __GNUC__ >= 10 - #pragma GCC diagnostic push - #pragma GCC diagnostic ignored "-Wstringop-overflow" -#endif - /// NOTE: This ID is used to create part names which are then persisted in ZK and as directory names on the file system. /// So if you want to change this method, be sure to guarantee compatibility with existing table data. String MergeTreePartition::getID(const Block & partition_key_sample) const @@ -92,9 +87,6 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const return result; } -#if defined (__GNUC__) && __GNUC__ >= 10 - #pragma GCC diagnostic pop -#endif void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const { From c36192db233af7ce3f971a0cd950db4cfbb6175d Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 11 Sep 2020 13:54:03 +0300 Subject: [PATCH 108/625] Remove diff --- src/Storages/MergeTree/MergeTreePartition.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 880a3aa181d..4a846f63b7c 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -87,7 +87,6 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const return result; } - void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); From ebb9de1376d50e834b61b48cc2f4695513244ad9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 11 Sep 2020 14:13:41 +0300 Subject: [PATCH 109/625] Supress strange warning --- src/Functions/negate.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Functions/negate.cpp b/src/Functions/negate.cpp index 39ca434ea89..3101513648b 100644 --- a/src/Functions/negate.cpp +++ b/src/Functions/negate.cpp @@ -13,7 +13,14 @@ struct NegateImpl static inline NO_SANITIZE_UNDEFINED ResultType apply(A a) { - return -static_cast(a); +#if defined (__GNUC__) && __GNUC__ >= 10 + #pragma GCC diagnostic push + #pragma GCC diagnostic ignored "-Wvector-operation-performance" +#endif + return -(static_cast(a)); +#if defined (__GNUC__) && __GNUC__ >= 10 + #pragma GCC diagnostic pop +#endif } #if USE_EMBEDDED_COMPILER From 5ce0c21bbe3c08a0f5169bced9dcea208857c88a Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 11 Sep 2020 14:24:42 +0300 Subject: [PATCH 110/625] Remove redundant change --- src/Functions/negate.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/negate.cpp b/src/Functions/negate.cpp index 3101513648b..de3995927d3 100644 --- a/src/Functions/negate.cpp +++ b/src/Functions/negate.cpp @@ -17,7 +17,7 @@ struct NegateImpl #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wvector-operation-performance" #endif - return -(static_cast(a)); + return -static_cast(a); #if defined (__GNUC__) && __GNUC__ >= 10 #pragma GCC diagnostic pop #endif From a64331d79f04bb9321383269150fe8302289e9b2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 11 Sep 2020 16:09:26 +0300 Subject: [PATCH 111/625] fix syncing of WAL --- src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp | 7 +++++++ src/Storages/MergeTree/MergeTreeWriteAheadLog.h | 2 ++ 2 files changed, 9 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index bc6738a8321..35fadb999b4 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -37,6 +37,13 @@ MergeTreeWriteAheadLog::MergeTreeWriteAheadLog( }); } +MergeTreeWriteAheadLog::~MergeTreeWriteAheadLog() +{ + std::unique_lock lock(write_mutex); + if (sync_scheduled) + sync_cv.wait(lock, [this] { return !sync_scheduled; }); +} + void MergeTreeWriteAheadLog::init() { out = disk->writeFile(path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append); diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index c5675eac696..77c7c7e11e7 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -35,6 +35,8 @@ public: MergeTreeWriteAheadLog(MergeTreeData & storage_, const DiskPtr & disk_, const String & name = DEFAULT_WAL_FILE_NAME); + ~MergeTreeWriteAheadLog(); + void addPart(const Block & block, const String & part_name); void dropPart(const String & part_name); std::vector restore(const StorageMetadataPtr & metadata_snapshot); From 7bbf7b295095cf6b9315ae9533b82d5ef9e519bc Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 11 Sep 2020 23:26:01 +0800 Subject: [PATCH 112/625] improvement chinese translation of remote.md --- .../sql-reference/table-functions/remote.md | 29 +++++++------------ 1 file changed, 11 insertions(+), 18 deletions(-) diff --git a/docs/zh/sql-reference/table-functions/remote.md b/docs/zh/sql-reference/table-functions/remote.md index 1125353e2fa..3ec1da3cd2c 100644 --- a/docs/zh/sql-reference/table-functions/remote.md +++ b/docs/zh/sql-reference/table-functions/remote.md @@ -1,13 +1,6 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 40 -toc_title: "\u8FDC\u7A0B" ---- - # 远程,远程安全 {#remote-remotesecure} -允许您访问远程服务器,而无需创建 `Distributed` 桌子 +允许您访问远程服务器,而无需创建 `Distributed` 表 签名: @@ -18,10 +11,10 @@ remoteSecure('addresses_expr', db, table[, 'user'[, 'password']]) remoteSecure('addresses_expr', db.table[, 'user'[, 'password']]) ``` -`addresses_expr` – An expression that generates addresses of remote servers. This may be just one server address. The server address is `host:port`,或者只是 `host`. 主机可以指定为服务器名称,也可以指定为IPv4或IPv6地址。 IPv6地址在方括号中指定。 端口是远程服务器上的TCP端口。 如果省略端口,它使用 `tcp_port` 从服务器的配置文件(默认情况下,9000)。 +`addresses_expr` – 代表远程服务器地址的一个表达式。可以只是单个服务器地址。 服务器地址可以是 `host:port` 或 `host`。`host` 可以指定为服务器域名,或是IPV4或IPV6地址。IPv6地址在方括号中指定。`port` 是远程服务器上的TCP端口。 如果省略端口,则使用服务器配置文件中的 `tcp_port` (默认情况为,9000)。 !!! important "重要事项" - IPv6地址需要该端口。 + IPv6地址需要指定端口。 例: @@ -34,7 +27,7 @@ localhost [2a02:6b8:0:1111::11]:9000 ``` -多个地址可以用逗号分隔。 在这种情况下,ClickHouse将使用分布式处理,因此它将将查询发送到所有指定的地址(如具有不同数据的分片)。 +多个地址可以用逗号分隔。在这种情况下,ClickHouse将使用分布式处理,因此它将将查询发送到所有指定的地址(如具有不同数据的分片)。 示例: @@ -56,7 +49,7 @@ example01-{01..02}-1 如果您有多对大括号,它会生成相应集合的直接乘积。 -大括号中的地址和部分地址可以用管道符号(\|)分隔。 在这种情况下,相应的地址集被解释为副本,并且查询将被发送到第一个正常副本。 但是,副本将按照当前设置的顺序进行迭代 [load\_balancing](../../operations/settings/settings.md) 设置。 +大括号中的地址和部分地址可以用管道符号(\|)分隔。 在这种情况下,相应的地址集被解释为副本,并且查询将被发送到第一个正常副本。 但是,副本将按照当前[load\_balancing](../../operations/settings/settings.md)设置的顺序进行迭代。 示例: @@ -66,20 +59,20 @@ example01-{01..02}-{1|2} 此示例指定两个分片,每个分片都有两个副本。 -生成的地址数由常量限制。 现在这是1000个地址。 +生成的地址数由常量限制。目前这是1000个地址。 -使用 `remote` 表函数比创建一个不太优化 `Distributed` 表,因为在这种情况下,服务器连接被重新建立为每个请求。 此外,如果设置了主机名,则会解析这些名称,并且在使用各种副本时不会计算错误。 在处理大量查询时,始终创建 `Distributed` 表的时间提前,不要使用 `remote` 表功能。 +使用 `remote` 表函数没有创建一个 `Distributed` 表更优,因为在这种情况下,将为每个请求重新建立服务器连接。此外,如果设置了主机名,则会解析这些名称,并且在使用各种副本时不会计算错误。 在处理大量查询时,始终优先创建 `Distributed` 表,不要使用 `remote` 表功能。 该 `remote` 表函数可以在以下情况下是有用的: - 访问特定服务器进行数据比较、调试和测试。 -- 查询之间的各种ClickHouse群集用于研究目的。 -- 手动发出的罕见分布式请求。 +- 在多个ClickHouse集群之间的用户研究目的的查询。 +- 手动发出的不频繁分布式请求。 - 每次重新定义服务器集的分布式请求。 -如果未指定用户, `default` 被使用。 +如果未指定用户, 将会使用`default`。 如果未指定密码,则使用空密码。 -`remoteSecure` -相同 `remote` but with secured connection. Default port — [tcp\_port\_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) 从配置或9440. +`remoteSecure` - 与 `remote` 相同,但是会使用加密链接。默认端口 — [tcp\_port\_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) 配置文件或或9440. [原始文章](https://clickhouse.tech/docs/en/query_language/table_functions/remote/) From d9394fbf66b5313d5c07bfc3d2e9119837516525 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 11 Sep 2020 18:51:08 +0300 Subject: [PATCH 113/625] Fix code --- src/Core/MultiEnum.h | 4 ++-- tests/ci/ci_config.json | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Core/MultiEnum.h b/src/Core/MultiEnum.h index 748550a8779..ddfc5b13e86 100644 --- a/src/Core/MultiEnum.h +++ b/src/Core/MultiEnum.h @@ -83,13 +83,13 @@ struct MultiEnum template >> friend bool operator==(ValueType left, MultiEnum right) { - return right == left; + return right.operator==(left); } template friend bool operator!=(L left, MultiEnum right) { - return !(right == left); + return !(right.operator==(left)); } private: diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index adb736a8df3..9a11a06db0d 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -92,7 +92,7 @@ "with_coverage": false }, { - "compiler": "gcc-10", + "compiler": "gcc-9", "build-type": "", "sanitizer": "", "package-type": "deb", From dd867b787f0de6d6d7dca46a6bcf451990ceed6d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 3 Sep 2020 01:35:47 +0300 Subject: [PATCH 114/625] 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 115/625] 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 1e849f297549f90bd7671286cace24f36c14e801 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 12 Sep 2020 03:16:50 +0300 Subject: [PATCH 116/625] Fix permission denied on opening file /var/lib/clickhouse/status in integration tests. --- tests/integration/helpers/cluster.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 44a22d3fe2e..a8704ee42b1 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1165,6 +1165,7 @@ class ClickHouseInstance: db_dir = p.abspath(p.join(self.path, 'database')) print "Setup database dir {}".format(db_dir) + os.mkdir(db_dir) if self.clickhouse_path_dir is not None: print "Database files taken from {}".format(self.clickhouse_path_dir) shutil.copytree(self.clickhouse_path_dir, db_dir) From 8242a948804622f71eeaba1ad91a6e1cd14ab683 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 12 Sep 2020 15:42:07 +0300 Subject: [PATCH 117/625] Update ci_config.json --- 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 9a11a06db0d..504b554029b 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -323,7 +323,7 @@ }, "Functional stateless tests (unbundled)": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "gcc-9", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", From 8075ce28099ea34f26209ab5eba7c8eb9bc603b2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 12 Sep 2020 15:42:32 +0300 Subject: [PATCH 118/625] Update warnings.cmake --- cmake/warnings.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 6b26b9b95a5..425972f00d8 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -23,7 +23,7 @@ option (WEVERYTHING "Enables -Weverything option with some exceptions. This is i # Control maximum size of stack frames. It can be important if the code is run in fibers with small stack size. # Only in release build because debug has too large stack frames. if ((NOT CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") AND (NOT SANITIZE)) - add_warning(frame-larger-than=32768) + add_warning(frame-larger-than=16384) endif () if (COMPILER_CLANG) From 7185b9a9452c958eca79e58f77873717b225343d Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Sat, 12 Sep 2020 19:59:03 +0300 Subject: [PATCH 119/625] 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 120/625] 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 421eeeccef7622f8f1462f9bce87303d51b880be Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 17 Aug 2020 17:38:10 +0300 Subject: [PATCH 121/625] Add the section user_directories to the default config. --- programs/server/config.xml | 16 +++++++++++----- .../helpers/0_common_instance_config.xml | 3 +++ 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index af01e880dc2..3d7ebf0cd96 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -212,8 +212,17 @@ /var/lib/clickhouse/user_files/ - - /var/lib/clickhouse/access/ + + + + + users.xml + + + + /var/lib/clickhouse/access/ + + @@ -256,9 +265,6 @@ --> - - users.xml - default diff --git a/tests/integration/helpers/0_common_instance_config.xml b/tests/integration/helpers/0_common_instance_config.xml index 5377efbc241..b27ecf0c3ef 100644 --- a/tests/integration/helpers/0_common_instance_config.xml +++ b/tests/integration/helpers/0_common_instance_config.xml @@ -4,4 +4,7 @@ custom_ /var/lib/clickhouse/ /var/lib/clickhouse/tmp/ + + + users.xml From 882b2a33488c4dbabbe96fff40c01a065fe0a860 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 13 Sep 2020 01:00:04 +0800 Subject: [PATCH 122/625] 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 c2d79bc5ccb04aeef881379797c05d57e290782b Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 10 Sep 2020 22:56:15 +0800 Subject: [PATCH 123/625] Add merge_algorithm to system.merges --- docs/en/operations/system-tables/merges.md | 6 ++++- src/Storages/MergeTree/MergeAlgorithm.cpp | 26 +++++++++++++++++++ src/Storages/MergeTree/MergeAlgorithm.h | 17 ++++++++++++ src/Storages/MergeTree/MergeList.cpp | 2 ++ src/Storages/MergeTree/MergeList.h | 3 +++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 7 ++--- .../MergeTree/MergeTreeDataMergerMutator.h | 7 +---- src/Storages/System/StorageSystemMerges.cpp | 7 +++++ src/Storages/ya.make | 1 + 9 files changed, 64 insertions(+), 12 deletions(-) create mode 100644 src/Storages/MergeTree/MergeAlgorithm.cpp create mode 100644 src/Storages/MergeTree/MergeAlgorithm.h diff --git a/docs/en/operations/system-tables/merges.md b/docs/en/operations/system-tables/merges.md index fb98a2b9e34..3e712e2962c 100644 --- a/docs/en/operations/system-tables/merges.md +++ b/docs/en/operations/system-tables/merges.md @@ -10,12 +10,16 @@ Columns: - `progress` (Float64) — The percentage of completed work from 0 to 1. - `num_parts` (UInt64) — The number of pieces to be merged. - `result_part_name` (String) — The name of the part that will be formed as the result of merging. -- `is_mutation` (UInt8) - 1 if this process is a part mutation. +- `is_mutation` (UInt8) — 1 if this process is a part mutation. - `total_size_bytes_compressed` (UInt64) — The total size of the compressed data in the merged chunks. - `total_size_marks` (UInt64) — The total number of marks in the merged parts. - `bytes_read_uncompressed` (UInt64) — Number of bytes read, uncompressed. - `rows_read` (UInt64) — Number of rows read. - `bytes_written_uncompressed` (UInt64) — Number of bytes written, uncompressed. - `rows_written` (UInt64) — Number of rows written. +- `memory_usage` (UInt64) — Memory consumption of the merge process. +- `thread_id` (UInt64) — Thread ID of the merge process. +- `merge_type` — The type of current merge. Empty if it's an mutation. +- `merge_algorithm` — The algorithm used in current merge. Empty if it's an mutation. [Original article](https://clickhouse.tech/docs/en/operations/system_tables/merges) diff --git a/src/Storages/MergeTree/MergeAlgorithm.cpp b/src/Storages/MergeTree/MergeAlgorithm.cpp new file mode 100644 index 00000000000..9f73557e701 --- /dev/null +++ b/src/Storages/MergeTree/MergeAlgorithm.cpp @@ -0,0 +1,26 @@ +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +String toString(MergeAlgorithm merge_algorithm) +{ + switch (merge_algorithm) + { + case MergeAlgorithm::Undecided: + return "Undecided"; + case MergeAlgorithm::Horizontal: + return "Horizontal"; + case MergeAlgorithm::Vertical: + return "Vertical"; + } + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown MergeAlgorithm {}", static_cast(merge_algorithm)); +} + +} diff --git a/src/Storages/MergeTree/MergeAlgorithm.h b/src/Storages/MergeTree/MergeAlgorithm.h new file mode 100644 index 00000000000..813767f9fb1 --- /dev/null +++ b/src/Storages/MergeTree/MergeAlgorithm.h @@ -0,0 +1,17 @@ +#pragma once + +#include + +namespace DB +{ +/// Algorithm of Merge. +enum class MergeAlgorithm +{ + Undecided, /// Not running yet + Horizontal, /// per-row merge of all columns + Vertical /// per-row merge of PK and secondary indices columns, per-column gather for non-PK columns +}; + +String toString(MergeAlgorithm merge_algorithm); + +} diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 30324bd5d9e..05d4cc6f963 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -24,6 +24,7 @@ MergeListElement::MergeListElement(const std::string & database_, const std::str , num_parts{future_part.parts.size()} , thread_id{getThreadId()} , merge_type{future_part.merge_type} + , merge_algorithm{MergeAlgorithm::Undecided} { for (const auto & source_part : future_part.parts) { @@ -74,6 +75,7 @@ MergeInfo MergeListElement::getInfo() const res.memory_usage = memory_tracker.get(); res.thread_id = thread_id; res.merge_type = toString(merge_type); + res.merge_algorithm = toString(merge_algorithm); for (const auto & source_part_name : source_part_names) res.source_part_names.emplace_back(source_part_name); diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 4d080ff3569..c1166c55703 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -47,6 +48,7 @@ struct MergeInfo UInt64 memory_usage; UInt64 thread_id; std::string merge_type; + std::string merge_algorithm; }; struct FutureMergedMutatedPart; @@ -90,6 +92,7 @@ struct MergeListElement : boost::noncopyable UInt64 thread_id; MergeType merge_type; + MergeAlgorithm merge_algorithm; MergeListElement(const std::string & database, const std::string & table, const FutureMergedMutatedPart & future_part); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index a8f7e265f68..99be79390be 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -62,10 +62,6 @@ namespace ErrorCodes extern const int ABORTED; } - -using MergeAlgorithm = MergeTreeDataMergerMutator::MergeAlgorithm; - - /// Do not start to merge parts, if free space is less than sum size of parts times specified coefficient. /// This value is chosen to not allow big merges to eat all free space. Thus allowing small merges to proceed. static const double DISK_USAGE_COEFFICIENT_TO_SELECT = 2; @@ -699,6 +695,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor size_t sum_input_rows_upper_bound = merge_entry->total_rows_count; MergeAlgorithm merge_alg = chooseMergeAlgorithm(parts, sum_input_rows_upper_bound, gathering_columns, deduplicate, need_remove_expired_values); + merge_entry->merge_algorithm = merge_alg; LOG_DEBUG(log, "Selected MergeAlgorithm: {}", ((merge_alg == MergeAlgorithm::Vertical) ? "Vertical" : "Horizontal")); @@ -1238,7 +1235,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor } -MergeTreeDataMergerMutator::MergeAlgorithm MergeTreeDataMergerMutator::chooseMergeAlgorithm( +MergeAlgorithm MergeTreeDataMergerMutator::chooseMergeAlgorithm( const MergeTreeData::DataPartsVector & parts, size_t sum_rows_upper_bound, const NamesAndTypesList & gathering_columns, bool deduplicate, bool need_remove_expired_values) const { diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 96ab14ba57b..2ba6b04e082 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -6,6 +6,7 @@ #include #include #include +#include #include @@ -226,12 +227,6 @@ public : ActionBlocker merges_blocker; ActionBlocker ttl_merges_blocker; - enum class MergeAlgorithm - { - Horizontal, /// per-row merge of all columns - Vertical /// per-row merge of PK and secondary indices columns, per-column gather for non-PK columns - }; - private: MergeAlgorithm chooseMergeAlgorithm( diff --git a/src/Storages/System/StorageSystemMerges.cpp b/src/Storages/System/StorageSystemMerges.cpp index 3b9e39c1ef8..b61324818e4 100644 --- a/src/Storages/System/StorageSystemMerges.cpp +++ b/src/Storages/System/StorageSystemMerges.cpp @@ -31,6 +31,7 @@ NamesAndTypesList StorageSystemMerges::getNamesAndTypes() {"memory_usage", std::make_shared()}, {"thread_id", std::make_shared()}, {"merge_type", std::make_shared()}, + {"merge_algorithm", std::make_shared()}, }; } @@ -67,9 +68,15 @@ void StorageSystemMerges::fillData(MutableColumns & res_columns, const Context & res_columns[i++]->insert(merge.memory_usage); res_columns[i++]->insert(merge.thread_id); if (!merge.is_mutation) + { res_columns[i++]->insert(merge.merge_type); + res_columns[i++]->insert(merge.merge_algorithm); + } else + { res_columns[i++]->insertDefault(); + res_columns[i++]->insertDefault(); + } } } diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 597e0c6f975..20377428857 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -36,6 +36,7 @@ SRCS( MergeTree/KeyCondition.cpp MergeTree/LevelMergeSelector.cpp MergeTree/localBackup.cpp + MergeTree/MergeAlgorithm.cpp MergeTree/MergedBlockOutputStream.cpp MergeTree/MergedColumnOnlyOutputStream.cpp MergeTree/MergeList.cpp From 016f707ea133f323ffd135a91ac86959112c6a8e Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 4 Sep 2020 01:51:16 +0800 Subject: [PATCH 124/625] column transformers in insert select --- src/Interpreters/InterpreterInsertQuery.cpp | 26 ++++++++++++++++++- src/Parsers/ParserInsertQuery.cpp | 9 ++++++- src/Parsers/ParserInsertQuery.h | 9 +++++++ ...1470_test_insert_select_asterisk.reference | 6 +++++ .../01470_test_insert_select_asterisk.sql | 18 +++++++++++++ 5 files changed, 66 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01470_test_insert_select_asterisk.reference create mode 100644 tests/queries/0_stateless/01470_test_insert_select_asterisk.sql diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 9d33650405a..01fee30a445 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -29,6 +30,8 @@ #include #include #include +#include +#include namespace { @@ -90,9 +93,30 @@ Block InterpreterInsertQuery::getSampleBlock( } Block table_sample = metadata_snapshot->getSampleBlock(); + const auto & columns = metadata_snapshot->getColumns(); + auto names_and_types = columns.getOrdinary(); + removeDuplicateColumns(names_and_types); + auto table_expr = std::make_shared(); + table_expr->database_and_table_name = createTableIdentifier(table->getStorageID()); + table_expr->children.push_back(table_expr->database_and_table_name); + TablesWithColumns tables_with_columns; + tables_with_columns.emplace_back(DatabaseAndTableWithAlias(*table_expr, context.getCurrentDatabase()), names_and_types); + + tables_with_columns[0].addHiddenColumns(columns.getMaterialized()); + tables_with_columns[0].addHiddenColumns(columns.getAliases()); + tables_with_columns[0].addHiddenColumns(table->getVirtuals()); + + NameSet source_columns_set; + for (const auto & identifier : query.columns->children) + source_columns_set.insert(identifier->getColumnName()); + TranslateQualifiedNamesVisitor::Data visitor_data(source_columns_set, tables_with_columns); + TranslateQualifiedNamesVisitor visitor(visitor_data); + auto columns_ast = query.columns->clone(); + visitor.visit(columns_ast); + /// Form the block based on the column names from the query Block res; - for (const auto & identifier : query.columns->children) + for (const auto & identifier : columns_ast->children) { std::string current_name = identifier->getColumnName(); diff --git a/src/Parsers/ParserInsertQuery.cpp b/src/Parsers/ParserInsertQuery.cpp index dc25954c71f..50baf7566d1 100644 --- a/src/Parsers/ParserInsertQuery.cpp +++ b/src/Parsers/ParserInsertQuery.cpp @@ -36,7 +36,7 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserToken s_lparen(TokenType::OpeningRoundBracket); ParserToken s_rparen(TokenType::ClosingRoundBracket); ParserIdentifier name_p; - ParserList columns_p(std::make_unique(), std::make_unique(TokenType::Comma), false); + ParserList columns_p(std::make_unique(), std::make_unique(TokenType::Comma), false); ParserFunction table_function_p{false}; ASTPtr database; @@ -189,5 +189,12 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return true; } +bool ParserInsertElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + return ParserColumnsMatcher().parse(pos, node, expected) + || ParserQualifiedAsterisk().parse(pos, node, expected) + || ParserAsterisk().parse(pos, node, expected) + || ParserCompoundIdentifier().parse(pos, node, expected); +} } diff --git a/src/Parsers/ParserInsertQuery.h b/src/Parsers/ParserInsertQuery.h index b69bc645c15..b6a199c9d71 100644 --- a/src/Parsers/ParserInsertQuery.h +++ b/src/Parsers/ParserInsertQuery.h @@ -33,4 +33,13 @@ public: ParserInsertQuery(const char * end_) : end(end_) {} }; +/** Insert accepts an identifier and an asterisk with variants. + */ +class ParserInsertElement : public IParserBase +{ +protected: + const char * getName() const override { return "insert element"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + } diff --git a/tests/queries/0_stateless/01470_test_insert_select_asterisk.reference b/tests/queries/0_stateless/01470_test_insert_select_asterisk.reference new file mode 100644 index 00000000000..c5d97af6937 --- /dev/null +++ b/tests/queries/0_stateless/01470_test_insert_select_asterisk.reference @@ -0,0 +1,6 @@ +1 0 0 2 +3 0 0 4 +1 0 0 2 +3 0 0 4 +1 0 0 2 +3 0 0 4 diff --git a/tests/queries/0_stateless/01470_test_insert_select_asterisk.sql b/tests/queries/0_stateless/01470_test_insert_select_asterisk.sql new file mode 100644 index 00000000000..607b8a25f82 --- /dev/null +++ b/tests/queries/0_stateless/01470_test_insert_select_asterisk.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS insert_select_dst; +DROP TABLE IF EXISTS insert_select_src; + +CREATE TABLE insert_select_dst (i int, middle_a int, middle_b int, j int) ENGINE = Log; + +CREATE TABLE insert_select_src (i int, j int) ENGINE = Log; + +INSERT INTO insert_select_src VALUES (1, 2), (3, 4); + +INSERT INTO insert_select_dst(* EXCEPT (middle_a, middle_b)) SELECT * FROM insert_select_src; +INSERT INTO insert_select_dst(insert_select_dst.* EXCEPT (middle_a, middle_b)) SELECT * FROM insert_select_src; +INSERT INTO insert_select_dst(COLUMNS('.*') EXCEPT (middle_a, middle_b)) SELECT * FROM insert_select_src; +INSERT INTO insert_select_dst(insert_select_src.* EXCEPT (middle_a, middle_b)) SELECT * FROM insert_select_src; -- { serverError 47 } + +SELECT * FROM insert_select_dst; + +DROP TABLE IF EXISTS insert_select_dst; +DROP TABLE IF EXISTS insert_select_src; From 34b9547ce1e51c729489f9555d6a60c8c8b7b078 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 5 Sep 2020 22:12:47 +0800 Subject: [PATCH 125/625] Binary operator monotonicity --- src/Functions/FunctionBinaryArithmetic.h | 186 +++++++++++++++++- src/Functions/bitAnd.cpp | 2 +- src/Functions/bitBoolMaskAnd.cpp | 2 +- src/Functions/bitBoolMaskOr.cpp | 2 +- src/Functions/bitOr.cpp | 2 +- src/Functions/bitRotateLeft.cpp | 2 +- src/Functions/bitRotateRight.cpp | 2 +- src/Functions/bitShiftLeft.cpp | 2 +- src/Functions/bitShiftRight.cpp | 2 +- src/Functions/bitTest.cpp | 2 +- src/Functions/bitXor.cpp | 2 +- src/Functions/divide.cpp | 2 +- src/Functions/gcd.cpp | 2 +- src/Functions/intDiv.cpp | 2 +- src/Functions/intDivOrZero.cpp | 2 +- src/Functions/lcm.cpp | 2 +- src/Functions/minus.cpp | 2 +- src/Functions/modulo.cpp | 2 +- src/Functions/moduloOrZero.cpp | 2 +- src/Functions/multiply.cpp | 2 +- src/Functions/plus.cpp | 2 +- src/Storages/MergeTree/KeyCondition.cpp | 42 +++- ...480_binary_operator_monotonicity.reference | 0 .../01480_binary_operator_monotonicity.sql | 10 + 24 files changed, 247 insertions(+), 31 deletions(-) create mode 100644 tests/queries/0_stateless/01480_binary_operator_monotonicity.reference create mode 100644 tests/queries/0_stateless/01480_binary_operator_monotonicity.sql diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index ca0cc876035..f30b564d677 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -28,6 +28,7 @@ #include "FunctionFactory.h" #include #include +#include #if !defined(ARCADIA_BUILD) # include @@ -51,6 +52,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int DECIMAL_OVERFLOW; extern const int CANNOT_ADD_DIFFERENT_AGGREGATE_STATES; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -602,7 +604,8 @@ class FunctionBinaryArithmetic : public IFunction return castType(left, [&](const auto & left_) { return castType(right, [&](const auto & right_) { return f(left_, right_); }); }); } - FunctionOverloadResolverPtr getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1) const + static FunctionOverloadResolverPtr + getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, const Context & context) { bool first_is_date_or_datetime = isDateOrDateTime(type0); bool second_is_date_or_datetime = isDateOrDateTime(type1); @@ -632,7 +635,7 @@ class FunctionBinaryArithmetic : public IFunction } if (second_is_date_or_datetime && is_minus) - throw Exception("Wrong order of arguments for function " + getName() + ": argument of type Interval cannot be first.", + throw Exception("Wrong order of arguments for function " + String(name) + ": argument of type Interval cannot be first.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); std::string function_name; @@ -651,7 +654,7 @@ class FunctionBinaryArithmetic : public IFunction return FunctionFactory::instance().get(function_name, context); } - bool isAggregateMultiply(const DataTypePtr & type0, const DataTypePtr & type1) const + static bool isAggregateMultiply(const DataTypePtr & type0, const DataTypePtr & type1) { if constexpr (!is_multiply) return false; @@ -663,7 +666,7 @@ class FunctionBinaryArithmetic : public IFunction || (which0.isNativeUInt() && which1.isAggregateFunction()); } - bool isAggregateAddition(const DataTypePtr & type0, const DataTypePtr & type1) const + static bool isAggregateAddition(const DataTypePtr & type0, const DataTypePtr & type1) { if constexpr (!is_plus) return false; @@ -812,6 +815,11 @@ public: size_t getNumberOfArguments() const override { return 2; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + return getReturnTypeImplStatic(arguments, context); + } + + static DataTypePtr getReturnTypeImplStatic(const DataTypes & arguments, const Context & context) { /// Special case when multiply aggregate function state if (isAggregateMultiply(arguments[0], arguments[1])) @@ -832,7 +840,7 @@ public: } /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. - if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0], arguments[1])) + if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0], arguments[1], context)) { ColumnsWithTypeAndName new_arguments(2); @@ -903,7 +911,7 @@ public: return false; }); if (!valid) - throw Exception("Illegal types " + arguments[0]->getName() + " and " + arguments[1]->getName() + " of arguments of function " + getName(), + throw Exception("Illegal types " + arguments[0]->getName() + " and " + arguments[1]->getName() + " of arguments of function " + String(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return type_res; } @@ -1110,7 +1118,8 @@ public: } /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. - if (auto function_builder = getFunctionForIntervalArithmetic(block.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type)) + if (auto function_builder + = getFunctionForIntervalArithmetic(block.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type, context)) { executeDateTimeIntervalPlusMinus(block, arguments, result, input_rows_count, function_builder); return; @@ -1200,4 +1209,167 @@ public: bool canBeExecutedOnDefaultArguments() const override { return valid_on_default_arguments; } }; + +template