From 3c0f6ce87ba6e424517b09b73bf11ec9bea84562 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Tue, 14 Aug 2018 12:57:18 +0300 Subject: [PATCH 01/79] ISSUES-2581 fix SELECT with select_sequential_consistency --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 29 ++++++-- .../MergeTree/MergeTreeDataSelectExecutor.h | 12 +++- .../MergeTree/StorageFromMergeTreeDataPart.h | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 67 ++++++++++++++++--- .../src/Storages/StorageReplicatedMergeTree.h | 6 ++ 6 files changed, 99 insertions(+), 19 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 93ee9220387..8f7196183ce 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -137,12 +137,26 @@ BlockInputStreams MergeTreeDataSelectExecutor::read( const Context & context, QueryProcessingStage::Enum & processed_stage, const size_t max_block_size, - const unsigned num_streams, - Int64 max_block_number_to_read) const + const unsigned num_streams) const { return readFromParts( data.getDataPartsVector(), column_names_to_return, query_info, context, processed_stage, - max_block_size, num_streams, max_block_number_to_read); + max_block_size, num_streams, std::unordered_map()); +} + + +BlockInputStreams MergeTreeDataSelectExecutor::read( + const Names & column_names_to_return, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + const size_t max_block_size, + const unsigned num_streams, + const std::unordered_map & max_blocks_number_to_read) const +{ + return readFromParts( + data.getDataPartsVector(), column_names_to_return, query_info, context, processed_stage, + max_block_size, num_streams, max_blocks_number_to_read); } BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( @@ -153,7 +167,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( QueryProcessingStage::Enum & processed_stage, const size_t max_block_size, const unsigned num_streams, - Int64 max_block_number_to_read) const + const std::unordered_map & max_blocks_number_to_read) const { size_t part_index = 0; @@ -270,7 +284,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( part->minmax_idx.parallelogram, data.minmax_idx_column_types)) continue; - if (max_block_number_to_read && part->info.max_block > max_block_number_to_read) + if (!max_blocks_number_to_read.empty() && max_blocks_number_to_read.find(part->info.partition_id) != max_blocks_number_to_read.end() + && part->info.max_block > max_blocks_number_to_read.at(part->info.partition_id)) continue; parts.push_back(part); @@ -323,7 +338,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( if (relative_sample_size == RelativeSize(1)) relative_sample_size = 0; - if (relative_sample_offset > 0 && RelativeSize(0) == relative_sample_size) + if (relative_sample_offset > 0 && 0 == relative_sample_size) throw Exception("Sampling offset is incorrect because no sampling", ErrorCodes::ARGUMENT_OUT_OF_BOUND); if (relative_sample_offset > 1) @@ -374,7 +389,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( if (!data.sampling_expression) throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED); - if (sample_factor_column_queried && relative_sample_size != RelativeSize(0)) + if (sample_factor_column_queried && relative_sample_size != 0) used_sample_factor = 1.0 / boost::rational_cast(relative_sample_size); RelativeSize size_of_universum = 0; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 96788cea015..c3d57ebc832 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -22,6 +22,14 @@ public: /** When reading, selects a set of parts that covers the desired range of the index. * max_block_number_to_read - if not zero, do not read all the parts whose right border is greater than this threshold. */ + BlockInputStreams read( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t max_block_size, + unsigned num_streams) const; + BlockInputStreams read( const Names & column_names, const SelectQueryInfo & query_info, @@ -29,7 +37,7 @@ public: QueryProcessingStage::Enum & processed_stage, size_t max_block_size, unsigned num_streams, - Int64 max_block_number_to_read) const; + const std::unordered_map & max_blocks_number_to_read) const; BlockInputStreams readFromParts( MergeTreeData::DataPartsVector parts, @@ -39,7 +47,7 @@ public: QueryProcessingStage::Enum & processed_stage, size_t max_block_size, unsigned num_streams, - Int64 max_block_number_to_read) const; + const std::unordered_map & max_blocks_number_to_read) const; private: MergeTreeData & data; diff --git a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index d7fffa2ac90..5eebc597eed 100644 --- a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -27,7 +27,7 @@ public: unsigned num_streams) override { return MergeTreeDataSelectExecutor(part->storage).readFromParts( - {part}, column_names, query_info, context, processed_stage, max_block_size, num_streams, 0); + {part}, column_names, query_info, context, processed_stage, max_block_size, num_streams, std::unordered_map()); } protected: diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 00e9f4c8697..9277b9bfd8e 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -114,7 +114,7 @@ BlockInputStreams StorageMergeTree::read( const size_t max_block_size, const unsigned num_streams) { - return reader.read(column_names, query_info, context, processed_stage, max_block_size, num_streams, 0); + return reader.read(column_names, query_info, context, processed_stage, max_block_size, num_streams); } BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & /*query*/, const Settings & /*settings*/) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 3718d56fcdb..b84dbff6281 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2615,8 +2615,17 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) zkutil::Requests ops; zkutil::Responses responses; + + /// We should get info about part for write it in zk node. + auto part_info = MergeTreePartInfo::fromPartName(part_name, data.format_version); + + zkutil::Stat last_part_stat; + String last_parts = zookeeper->get(quorum_last_part_path, &last_part_stat); + + String new_last_parts = rewriteLastParts(last_parts, part_info.partition_id, part_info.max_block); + ops.emplace_back(zkutil::makeRemoveRequest(quorum_status_path, stat.version)); - ops.emplace_back(zkutil::makeSetRequest(quorum_last_part_path, part_name, -1)); + ops.emplace_back(zkutil::makeSetRequest(quorum_last_part_path, part_name, last_part_stat.version)); auto code = zookeeper->tryMulti(ops, responses); if (code == ZooKeeperImpl::ZooKeeper::ZOK) @@ -2662,6 +2671,49 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) } +std::unordered_map StorageReplicatedMergeTree::getLastPartsWithQuorum(const String & last_parts_str) +{ + auto in = ReadBufferFromString(last_parts_str); + + size_t count; + std::unordered_map last_blocks; + + if (!last_parts_str.empty()) + { + in >> "parts_count " >> count >> "\n"; + for (size_t i = 0; i < count; ++i) + { + String part_id; + Int64 max_block; + in >> part_id >> "\t" >> max_block >> "\n"; + last_blocks[part_id] = max_block; + } + } + + return last_blocks; +} + + +String StorageReplicatedMergeTree::rewriteLastParts(const String & old_last_parts, const String & part_id, const Int64 & new_block) +{ + auto last_blocks = getLastPartsWithQuorum(old_last_parts); + + last_blocks[part_id] = new_block; + + String new_old_last_parts; + auto out = WriteBufferFromString(new_old_last_parts); + + out << "parts_count " << last_blocks.size() << "\n"; + + for (auto & part_info : last_blocks) + { + out << part_info.first << "\t" << part_info.second << "\n"; + } + + return new_old_last_parts; +} + + bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & replica_path, bool to_detached, size_t quorum) { if (auto part = data.getPartIfExists(part_name, {MergeTreeDataPart::State::Outdated, MergeTreeDataPart::State::Deleting})) @@ -2835,7 +2887,7 @@ BlockInputStreams StorageReplicatedMergeTree::read( * 2. Do not read parts that have not yet been written to the quorum of the replicas. * For this you have to synchronously go to ZooKeeper. */ - Int64 max_block_number_to_read = 0; + std::unordered_map max_blocks_number_to_read; if (settings.select_sequential_consistency) { auto zookeeper = getZooKeeper(); @@ -2843,11 +2895,12 @@ BlockInputStreams StorageReplicatedMergeTree::read( String last_part; zookeeper->tryGet(zookeeper_path + "/quorum/last_part", last_part); + /* if (!last_part.empty() && !data.getActiveContainingPart(last_part)) /// TODO Disable replica for distributed queries. throw Exception("Replica doesn't have part " + last_part + " which was successfully written to quorum of other replicas." " Send query to another replica or disable 'select_sequential_consistency' setting.", ErrorCodes::REPLICA_IS_NOT_IN_QUORUM); - - if (last_part.empty()) /// If no part has been written with quorum. + */ + if (last_parts.empty()) /// If no part has been written with quorum. { String quorum_str; if (zookeeper->tryGet(zookeeper_path + "/quorum/status", quorum_str)) @@ -2855,18 +2908,16 @@ BlockInputStreams StorageReplicatedMergeTree::read( ReplicatedMergeTreeQuorumEntry quorum_entry; quorum_entry.fromString(quorum_str); auto part_info = MergeTreePartInfo::fromPartName(quorum_entry.part_name, data.format_version); - max_block_number_to_read = part_info.min_block - 1; } } else { - auto part_info = MergeTreePartInfo::fromPartName(last_part, data.format_version); - max_block_number_to_read = part_info.max_block; + max_blocks_number_to_read = getLastPartsWithQuorum(last_parts); } } return reader.read( - column_names, query_info, context, processed_stage, max_block_size, num_streams, max_block_number_to_read); + column_names, query_info, context, processed_stage, max_block_size, num_streams, max_blocks_number_to_read); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 61dceb0e408..6f89a07c8de 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -404,6 +404,12 @@ private: */ bool queueTask(); + /// Get parts, wich were wrote with quorum. + std::unordered_map getLastPartsWithQuorum(const String & last_parts_str); + + /// Rewrite last parts with quorum + String rewriteLastParts(const String & old_last_part, const String & tmp, const Int64 & new_block); + /// Postcondition: /// either leader_election is fully initialized (node in ZK is created and the watching thread is launched) /// or an exception is thrown and leader_election is destroyed. From 537a8f02406266955f15434879db25720a72610d Mon Sep 17 00:00:00 2001 From: Vadim Date: Tue, 14 Aug 2018 13:03:32 +0300 Subject: [PATCH 02/79] Update MergeTreeDataSelectExecutor.cpp --- dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 8f7196183ce..a07f6f9995a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -338,7 +338,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( if (relative_sample_size == RelativeSize(1)) relative_sample_size = 0; - if (relative_sample_offset > 0 && 0 == relative_sample_size) + if (relative_sample_offset > 0 && RelativeSize(0) == relative_sample_size) throw Exception("Sampling offset is incorrect because no sampling", ErrorCodes::ARGUMENT_OUT_OF_BOUND); if (relative_sample_offset > 1) @@ -389,7 +389,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( if (!data.sampling_expression) throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED); - if (sample_factor_column_queried && relative_sample_size != 0) + if (sample_factor_column_queried && relative_sample_size != RelativeSize(0)) used_sample_factor = 1.0 / boost::rational_cast(relative_sample_size); RelativeSize size_of_universum = 0; From a37ad50329c35bc264e6e68f4eb3f64d136b5712 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Tue, 14 Aug 2018 13:24:39 +0300 Subject: [PATCH 03/79] ISSUES-2581 fix build --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index b84dbff6281..83df5289ca5 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2892,8 +2892,8 @@ BlockInputStreams StorageReplicatedMergeTree::read( { auto zookeeper = getZooKeeper(); - String last_part; - zookeeper->tryGet(zookeeper_path + "/quorum/last_part", last_part); + String last_parts; + zookeeper->tryGet(zookeeper_path + "/quorum/last_part", last_parts); /* if (!last_part.empty() && !data.getActiveContainingPart(last_part)) /// TODO Disable replica for distributed queries. From c8e2506b5f1cd13e8e1c79390d74a932c6df2532 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Tue, 14 Aug 2018 19:40:51 +0300 Subject: [PATCH 04/79] ISSUES-2581 add check when replica do not have part added with quorum --- .../Storages/StorageReplicatedMergeTree.cpp | 57 +++++++++++-------- .../src/Storages/StorageReplicatedMergeTree.h | 6 +- 2 files changed, 37 insertions(+), 26 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 83df5289ca5..b7987981fb8 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2615,14 +2615,11 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) zkutil::Requests ops; zkutil::Responses responses; - - /// We should get info about part for write it in zk node. - auto part_info = MergeTreePartInfo::fromPartName(part_name, data.format_version); zkutil::Stat last_part_stat; String last_parts = zookeeper->get(quorum_last_part_path, &last_part_stat); - String new_last_parts = rewriteLastParts(last_parts, part_info.partition_id, part_info.max_block); + String new_last_parts = rewriteLastParts(last_parts, part_name); ops.emplace_back(zkutil::makeRemoveRequest(quorum_status_path, stat.version)); ops.emplace_back(zkutil::makeSetRequest(quorum_last_part_path, part_name, last_part_stat.version)); @@ -2671,22 +2668,33 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) } -std::unordered_map StorageReplicatedMergeTree::getLastPartsWithQuorum(const String & last_parts_str) +std::unordered_map StorageReplicatedMergeTree::getMaxBlocksWithQuorum(const std::unordered_map & last_parts) +{ + std::unordered_map max_blocks; + for (auto last_part : last_parts) + { + max_blocks[last_part.first] = MergeTreePartInfo::fromPartName(last_part.second, data.format_version).max_block; + } + return max_blocks; +} + + +std::unordered_map StorageReplicatedMergeTree::getLastPartsWithQuorum(const String & last_parts_str) { auto in = ReadBufferFromString(last_parts_str); size_t count; - std::unordered_map last_blocks; + std::unordered_map last_blocks; if (!last_parts_str.empty()) { in >> "parts_count " >> count >> "\n"; for (size_t i = 0; i < count; ++i) { - String part_id; - Int64 max_block; - in >> part_id >> "\t" >> max_block >> "\n"; - last_blocks[part_id] = max_block; + String partition_id; + String part_name; + in >> partition_id >> "\t" >> part_name >> "\n"; + last_blocks[partition_id] = part_name; } } @@ -2694,23 +2702,25 @@ std::unordered_map StorageReplicatedMergeTree::getLastPartsWithQu } -String StorageReplicatedMergeTree::rewriteLastParts(const String & old_last_parts, const String & part_id, const Int64 & new_block) +String StorageReplicatedMergeTree::rewriteLastParts(const String & old_last_parts, const String & new_part_name) { auto last_blocks = getLastPartsWithQuorum(old_last_parts); - last_blocks[part_id] = new_block; + auto new_part_info = MergeTreePartInfo::fromPartName(new_part_name, data.format_version); - String new_old_last_parts; - auto out = WriteBufferFromString(new_old_last_parts); + last_blocks[new_part_info.partition_id] = new_part_name; + + String new_last_parts; + auto out = WriteBufferFromString(new_last_parts); out << "parts_count " << last_blocks.size() << "\n"; - for (auto & part_info : last_blocks) + for (auto & last_block : last_blocks) { - out << part_info.first << "\t" << part_info.second << "\n"; + out << last_block.first << "\t" << last_block.second << "\n"; } - return new_old_last_parts; + return new_last_parts; } @@ -2895,12 +2905,7 @@ BlockInputStreams StorageReplicatedMergeTree::read( String last_parts; zookeeper->tryGet(zookeeper_path + "/quorum/last_part", last_parts); - /* - if (!last_part.empty() && !data.getActiveContainingPart(last_part)) /// TODO Disable replica for distributed queries. - throw Exception("Replica doesn't have part " + last_part + " which was successfully written to quorum of other replicas." - " Send query to another replica or disable 'select_sequential_consistency' setting.", ErrorCodes::REPLICA_IS_NOT_IN_QUORUM); - */ - if (last_parts.empty()) /// If no part has been written with quorum. + if (last_parts.empty()) /// If no part has been written with quorum. { String quorum_str; if (zookeeper->tryGet(zookeeper_path + "/quorum/status", quorum_str)) @@ -2912,7 +2917,11 @@ BlockInputStreams StorageReplicatedMergeTree::read( } else { - max_blocks_number_to_read = getLastPartsWithQuorum(last_parts); + auto last_blocks_with_quorum = getLastPartsWithQuorum(last_parts); + for (auto & part_info : last_blocks_with_quorum) + if (!data.getActiveContainingPart(part_info.second)) + throw Exception("Replica doesn't have part " + part_info.second + " which was successfully written to quorum of other replicas." + " Send query to another replica or disable 'select_sequential_consistency' setting.", ErrorCodes::REPLICA_IS_NOT_IN_QUORUM); } } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 6f89a07c8de..2d71eec5c36 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -404,11 +404,13 @@ private: */ bool queueTask(); + std::unordered_map getMaxBlocksWithQuorum(const std::unordered_map & last_parts); + /// Get parts, wich were wrote with quorum. - std::unordered_map getLastPartsWithQuorum(const String & last_parts_str); + std::unordered_map getLastPartsWithQuorum(const String & last_parts_str); /// Rewrite last parts with quorum - String rewriteLastParts(const String & old_last_part, const String & tmp, const Int64 & new_block); + String rewriteLastParts(const String & old_last_parts, const String & new_part_name); /// Postcondition: /// either leader_election is fully initialized (node in ZK is created and the watching thread is launched) From 854192e6665fbc348028605532c218494be0b5f4 Mon Sep 17 00:00:00 2001 From: Vadim Date: Tue, 21 Aug 2018 17:26:20 +0300 Subject: [PATCH 05/79] Update StorageReplicatedMergeTree.cpp --- .../Storages/StorageReplicatedMergeTree.cpp | 24 +++++++++++++------ 1 file changed, 17 insertions(+), 7 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 3e94f56478c..b46c29818c1 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2690,13 +2690,23 @@ std::unordered_map StorageReplicatedMergeTree::getLastPartsWithQ if (!last_parts_str.empty()) { - in >> "parts_count " >> count >> "\n"; - for (size_t i = 0; i < count; ++i) + try { - String partition_id; - String part_name; - in >> partition_id >> "\t" >> part_name >> "\n"; - last_blocks[partition_id] = part_name; + in >> "parts_count " >> count >> "\n"; + for (size_t i = 0; i < count; ++i) + { + String partition_id; + String part_name; + in >> partition_id >> "\t" >> part_name >> "\n"; + last_blocks[partition_id] = part_name; + } + } + catch (Exception e) + { + String last_part; + in >> last_part; + auto partition_info = MergeTreePartInfo::fromPartName(last_part, data.format_version); + last_blocks[partition_info.partition_id] = last_part; } } @@ -2927,7 +2937,7 @@ BlockInputStreams StorageReplicatedMergeTree::read( " Send query to another replica or disable 'select_sequential_consistency' setting.", ErrorCodes::REPLICA_IS_NOT_IN_QUORUM); } } - + return reader.read(column_names, query_info, context, max_block_size, num_streams, max_blocks_number_to_read); } From 306c1b0b4839a3267686059645420cbfd497996b Mon Sep 17 00:00:00 2001 From: Vadim Date: Tue, 21 Aug 2018 17:27:51 +0300 Subject: [PATCH 06/79] Update MergeTreeDataSelectExecutor.h --- .../Storages/MergeTree/MergeTreeDataSelectExecutor.h | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 9276e9aa6ab..298c5a1fe18 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -20,15 +20,8 @@ public: MergeTreeDataSelectExecutor(MergeTreeData & data_); /** When reading, selects a set of parts that covers the desired range of the index. - * max_block_number_to_read - if not zero, do not read all the parts whose right border is greater than this threshold. - */ - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - size_t max_block_size, - unsigned num_streams) const; - + * max_blocks_number_to_read - if not empty, do not read all the parts whose right border is greater than max_block in partition. + */ BlockInputStreams read( const Names & column_names, const SelectQueryInfo & query_info, From b96d15f26636241dc270d63c907a529237cd7a6d Mon Sep 17 00:00:00 2001 From: Vadim Date: Tue, 21 Aug 2018 17:28:22 +0300 Subject: [PATCH 07/79] Update MergeTreeDataSelectExecutor.cpp --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 10dc32920bf..de3d441a1de 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -131,19 +131,6 @@ static RelativeSize convertAbsoluteSampleSizeToRelative(const ASTPtr & node, siz } -BlockInputStreams MergeTreeDataSelectExecutor::read( - const Names & column_names_to_return, - const SelectQueryInfo & query_info, - const Context & context, - const size_t max_block_size, - const unsigned num_streams) const -{ - return readFromParts( - data.getDataPartsVector(), column_names_to_return, query_info, context, processed_stage, - max_block_size, num_streams, std::unordered_map()); -} - - BlockInputStreams MergeTreeDataSelectExecutor::read( const Names & column_names_to_return, const SelectQueryInfo & query_info, From 5f7117e6a27f4b094a23c363f8cb203d75bbffbb Mon Sep 17 00:00:00 2001 From: Vadim Date: Tue, 21 Aug 2018 17:32:28 +0300 Subject: [PATCH 08/79] Update MergeTreeDataSelectExecutor.cpp --- dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index de3d441a1de..32a785b0589 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -135,7 +135,6 @@ BlockInputStreams MergeTreeDataSelectExecutor::read( const Names & column_names_to_return, const SelectQueryInfo & query_info, const Context & context, - QueryProcessingStage::Enum & processed_stage, const size_t max_block_size, const unsigned num_streams, const std::unordered_map & max_blocks_number_to_read) const From 79a089a996d0f4cfcda73eae7f5bac0d5df31070 Mon Sep 17 00:00:00 2001 From: Vadim Date: Tue, 21 Aug 2018 17:32:47 +0300 Subject: [PATCH 09/79] Update MergeTreeDataSelectExecutor.h --- dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 298c5a1fe18..dc8420e50be 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -26,7 +26,6 @@ public: const Names & column_names, const SelectQueryInfo & query_info, const Context & context, - QueryProcessingStage::Enum & processed_stage, size_t max_block_size, unsigned num_streams, const std::unordered_map & max_blocks_number_to_read) const; From e9954d3ab7c5f898d241e910b3612a39fa84ddfc Mon Sep 17 00:00:00 2001 From: Vadim Date: Tue, 21 Aug 2018 17:53:16 +0300 Subject: [PATCH 10/79] Update MergeTreeDataSelectExecutor.cpp --- dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 32a785b0589..39a1d6a136e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -141,7 +141,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read( { return readFromParts( data.getDataPartsVector(), column_names_to_return, query_info, context, - max_block_size, num_streams, max_block_number_to_read); + max_block_size, num_streams, max_blocks_number_to_read); } BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( From 6e78f576d79eaa77205d415e7ed31a55b1c79a1e Mon Sep 17 00:00:00 2001 From: VadimPE Date: Mon, 3 Sep 2018 11:25:35 +0300 Subject: [PATCH 11/79] ISSUES-2581 add comments, and fix --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- .../MergeTree/MergeTreeDataSelectExecutor.h | 8 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 90 +++++++++---------- .../src/Storages/StorageReplicatedMergeTree.h | 16 ++-- 6 files changed, 63 insertions(+), 59 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 39a1d6a136e..b070d10d723 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -137,7 +137,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read( const Context & context, const size_t max_block_size, const unsigned num_streams, - const std::unordered_map & max_blocks_number_to_read) const + const MaxAddedBlocks & max_blocks_number_to_read) const { return readFromParts( data.getDataPartsVector(), column_names_to_return, query_info, context, @@ -151,7 +151,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( const Context & context, const size_t max_block_size, const unsigned num_streams, - const std::unordered_map & max_blocks_number_to_read) const + const MaxAddedBlocks & max_blocks_number_to_read) const { size_t part_index = 0; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index dc8420e50be..6e727229f8f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -21,14 +21,16 @@ public: /** When reading, selects a set of parts that covers the desired range of the index. * max_blocks_number_to_read - if not empty, do not read all the parts whose right border is greater than max_block in partition. - */ + */ + using MaxAddedBlocks = std::unordered_map; + BlockInputStreams read( const Names & column_names, const SelectQueryInfo & query_info, const Context & context, size_t max_block_size, unsigned num_streams, - const std::unordered_map & max_blocks_number_to_read) const; + const MaxAddedBlocks & max_blocks_number_to_read) const; BlockInputStreams readFromParts( MergeTreeData::DataPartsVector parts, @@ -37,7 +39,7 @@ public: const Context & context, size_t max_block_size, unsigned num_streams, - const std::unordered_map & max_blocks_number_to_read) const; + const MaxAddedBlocks & max_blocks_number_to_read) const; private: MergeTreeData & data; diff --git a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index f4afa74012e..8e00d7045fe 100644 --- a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -27,7 +27,7 @@ public: unsigned num_streams) override { return MergeTreeDataSelectExecutor(part->storage).readFromParts( - {part}, column_names, query_info, context, max_block_size, num_streams, std::unordered_map()); + {part}, column_names, query_info, context, max_block_size, num_streams, {}); } protected: diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 184549e743f..b27865ce9ce 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -115,7 +115,7 @@ BlockInputStreams StorageMergeTree::read( const unsigned num_streams) { checkQueryProcessingStage(processed_stage, context); - return reader.read(column_names, query_info, context, max_block_size, num_streams, std::unordered_map()); + return reader.read(column_names, query_info, context, max_block_size, num_streams, {}); } BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & /*query*/, const Settings & /*settings*/) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 51974f79464..8135304b2e1 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2615,13 +2615,13 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) Coordination::Requests ops; Coordination::Responses responses; - Coordination::Stat last_part_stat; - String last_parts = zookeeper->get(quorum_last_part_path, &last_part_stat); + Coordination::Stat added_parts_stat; + String old_added_parts = zookeeper->get(quorum_last_part_path, &added_parts_stat); - String new_last_parts = rewriteLastParts(last_parts, part_name); + String new_added_parts = rewriteAddedParts(old_added_parts, part_name); ops.emplace_back(zkutil::makeRemoveRequest(quorum_status_path, stat.version)); - ops.emplace_back(zkutil::makeSetRequest(quorum_last_part_path, part_name, last_part_stat.version)); + ops.emplace_back(zkutil::makeSetRequest(quorum_last_part_path, new_added_parts, added_parts_stat.version)); auto code = zookeeper->tryMulti(ops, responses); if (code == Coordination::ZOK) @@ -2667,25 +2667,29 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) } -std::unordered_map StorageReplicatedMergeTree::getMaxBlocksWithQuorum(const std::unordered_map & last_parts) +using MaxAddedBlocks = std::unordered_map; +using PartsNames = std::unordered_map; + + +MaxAddedBlocks StorageReplicatedMergeTree::getMaxAddedBlocksWithQuorum(const PartsNames & parts) { - std::unordered_map max_blocks; - for (auto last_part : last_parts) + MaxAddedBlocks max_added_blocks; + for (auto & part : parts) { - max_blocks[last_part.first] = MergeTreePartInfo::fromPartName(last_part.second, data.format_version).max_block; + max_added_blocks[part.first] = MergeTreePartInfo::fromPartName(part.second, data.format_version).max_block; } - return max_blocks; + return max_added_blocks; } -std::unordered_map StorageReplicatedMergeTree::getLastPartsWithQuorum(const String & last_parts_str) +PartsNames StorageReplicatedMergeTree::getAddedPartsWithQuorum(const String & parts_str) { - auto in = ReadBufferFromString(last_parts_str); + auto in = ReadBufferFromString(parts_str); size_t count; - std::unordered_map last_blocks; + PartsNames added_parts; - if (!last_parts_str.empty()) + if (!parts_str.empty()) { try { @@ -2695,41 +2699,41 @@ std::unordered_map StorageReplicatedMergeTree::getLastPartsWithQ String partition_id; String part_name; in >> partition_id >> "\t" >> part_name >> "\n"; - last_blocks[partition_id] = part_name; + added_parts[partition_id] = part_name; } } catch (Exception e) { - String last_part; - in >> last_part; - auto partition_info = MergeTreePartInfo::fromPartName(last_part, data.format_version); - last_blocks[partition_info.partition_id] = last_part; + String last_added_part; + in >> last_added_part; + auto partition_info = MergeTreePartInfo::fromPartName(last_added_part, data.format_version); + added_parts[partition_info.partition_id] = last_added_part; } } - return last_blocks; + return added_parts; } -String StorageReplicatedMergeTree::rewriteLastParts(const String & old_last_parts, const String & new_part_name) +String StorageReplicatedMergeTree::rewriteAddedParts(const String & old_added_parts_str, const String & new_part_name) { - auto last_blocks = getLastPartsWithQuorum(old_last_parts); + auto added_parts = getAddedPartsWithQuorum(old_added_parts_str); auto new_part_info = MergeTreePartInfo::fromPartName(new_part_name, data.format_version); - last_blocks[new_part_info.partition_id] = new_part_name; + added_parts[new_part_info.partition_id] = new_part_name; - String new_last_parts; - auto out = WriteBufferFromString(new_last_parts); + String new_parts; + auto out = WriteBufferFromString(new_parts); - out << "parts_count " << last_blocks.size() << "\n"; + out << "parts_count " << added_parts.size() << "\n"; - for (auto & last_block : last_blocks) + for (auto & added_part : added_parts) { - out << last_block.first << "\t" << last_block.second << "\n"; + out << added_part.first << "\t" << added_part.second << "\n"; } - return new_last_parts; + return new_parts; } @@ -2915,35 +2919,27 @@ BlockInputStreams StorageReplicatedMergeTree::read( * 2. Do not read parts that have not yet been written to the quorum of the replicas. * For this you have to synchronously go to ZooKeeper. */ - std::unordered_map max_blocks_number_to_read; + MaxAddedBlocks max_added_blocks_with_quorum; if (settings.select_sequential_consistency) { auto zookeeper = getZooKeeper(); - String last_parts; - zookeeper->tryGet(zookeeper_path + "/quorum/last_part", last_parts); + String added_parts_str; + zookeeper->tryGet(zookeeper_path + "/quorum/last_part", added_parts_str); - if (last_parts.empty()) /// If no part has been written with quorum. + if (!added_parts_str.empty()) { - String quorum_str; - if (zookeeper->tryGet(zookeeper_path + "/quorum/status", quorum_str)) - { - ReplicatedMergeTreeQuorumEntry quorum_entry; - quorum_entry.fromString(quorum_str); - auto part_info = MergeTreePartInfo::fromPartName(quorum_entry.part_name, data.format_version); - } - } - else - { - auto last_blocks_with_quorum = getLastPartsWithQuorum(last_parts); - for (auto & part_info : last_blocks_with_quorum) - if (!data.getActiveContainingPart(part_info.second)) - throw Exception("Replica doesn't have part " + part_info.second + " which was successfully written to quorum of other replicas." + auto added_parts = getAddedPartsWithQuorum(added_parts_str); + for (auto & added_part : added_parts) + if (!data.getActiveContainingPart(added_part.second)) + throw Exception("Replica doesn't have part " + added_part.second + " which was successfully written to quorum of other replicas." " Send query to another replica or disable 'select_sequential_consistency' setting.", ErrorCodes::REPLICA_IS_NOT_IN_QUORUM); + + max_added_blocks_with_quorum = getMaxAddedBlocksWithQuorum(added_parts); } } - return reader.read(column_names, query_info, context, max_block_size, num_streams, max_blocks_number_to_read); + return reader.read(column_names, query_info, context, max_block_size, num_streams, max_added_blocks_with_quorum); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 2a869a99790..acec796c872 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -403,13 +403,19 @@ private: */ bool queueTask(); - std::unordered_map getMaxBlocksWithQuorum(const std::unordered_map & last_parts); + using MaxAddedBlocks = std::unordered_map; + using PartsNames = std::unordered_map; + + /// Return map with partition_id and max added block in this partition. + /// This is used to filter the added blocks in read(). + MaxAddedBlocks getMaxAddedBlocksWithQuorum(const PartsNames & parts); - /// Get parts, wich were wrote with quorum. - std::unordered_map getLastPartsWithQuorum(const String & last_parts_str); + /// Return map with partition_id and the last added part_name in this partition. + PartsNames getAddedPartsWithQuorum(const String & parts_str); - /// Rewrite last parts with quorum - String rewriteLastParts(const String & old_last_parts, const String & new_part_name); + /// Rewrite data with inserted partiton with quorum. + /// This is added new partition name in data for ZooKeeper. + String rewriteAddedParts(const String & old_added_parts_str, const String & new_added_part_name); /// Postcondition: /// either leader_election is fully initialized (node in ZK is created and the watching thread is launched) From 5defbadb4a0f674a1c3f902f320344443647fe69 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Mon, 17 Sep 2018 15:28:29 +0300 Subject: [PATCH 12/79] Add new class for write new parts --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- .../MergeTree/MergeTreeDataSelectExecutor.h | 8 +- .../ReplicatedMergeTreeQuorumWriter.h | 113 ++++++++++++++++++ .../Storages/StorageReplicatedMergeTree.cpp | 83 ++----------- .../src/Storages/StorageReplicatedMergeTree.h | 16 +-- 5 files changed, 130 insertions(+), 94 deletions(-) create mode 100644 dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index b070d10d723..9ccfffd034a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -137,7 +137,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read( const Context & context, const size_t max_block_size, const unsigned num_streams, - const MaxAddedBlocks & max_blocks_number_to_read) const + const PartitionIdToMaxBlock & max_blocks_number_to_read) const { return readFromParts( data.getDataPartsVector(), column_names_to_return, query_info, context, @@ -151,7 +151,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( const Context & context, const size_t max_block_size, const unsigned num_streams, - const MaxAddedBlocks & max_blocks_number_to_read) const + const PartitionIdToMaxBlock & max_blocks_number_to_read) const { size_t part_index = 0; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 6e727229f8f..9da2a95bd9a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -22,15 +22,15 @@ public: /** When reading, selects a set of parts that covers the desired range of the index. * max_blocks_number_to_read - if not empty, do not read all the parts whose right border is greater than max_block in partition. */ - using MaxAddedBlocks = std::unordered_map; - + using PartitionIdToMaxBlock = std::unordered_map; + BlockInputStreams read( const Names & column_names, const SelectQueryInfo & query_info, const Context & context, size_t max_block_size, unsigned num_streams, - const MaxAddedBlocks & max_blocks_number_to_read) const; + const PartitionIdToMaxBlock & max_blocks_number_to_read) const; BlockInputStreams readFromParts( MergeTreeData::DataPartsVector parts, @@ -39,7 +39,7 @@ public: const Context & context, size_t max_block_size, unsigned num_streams, - const MaxAddedBlocks & max_blocks_number_to_read) const; + const PartitionIdToMaxBlock & max_blocks_number_to_read) const; private: MergeTreeData & data; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h new file mode 100644 index 00000000000..85ed1cd39b7 --- /dev/null +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h @@ -0,0 +1,113 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +struct ReplicatedMergeTreeQuorumWriter +{ + using PartitionIdToMaxBlock = std::unordered_map; + using PartitonIdToPartName= std::unordered_map; + + PartitonIdToPartName added_parts; + + MergeTreeDataFormatVersion format_version; + + ReplicatedMergeTreeQuorumWriter(const std::string & old_added_parts, MergeTreeDataFormatVersion format_version_) + : format_version(format_version_) + { + read(old_added_parts); + } + + std::string write(const std::string & part_name) + { + WriteBufferFromOwnString out; + + auto partition_info = MergeTreePartInfo::fromPartName(part_name, format_version); + added_parts[partition_info.partition_id] = part_name; + + out << "parts_count " << added_parts.size() << '\n'; + + for (const auto & part : added_parts) + out << part.first << '\t' << part.second << '\n'; + + return out.str(); + } + + PartitonIdToPartName readParts() + { + return added_parts; + } + + PartitionIdToMaxBlock readBlocks() + { + PartitionIdToMaxBlock max_added_blocks; + + for (const auto & part : added_parts) + { + auto partition_info = MergeTreePartInfo::fromPartName(part.second, format_version); + max_added_blocks[part.first] = partition_info.max_block; + } + return max_added_blocks; + } + + void read(const std::string & str) + { + ReadBufferFromString in(str); + if (checkString("parts_count ", in)) + { + added_parts = read_v2(in); + } + else + added_parts = read_v3(in); + } + + PartitonIdToPartName read_v2(ReadBufferFromString & in) + { + PartitonIdToPartName parts_in_quorum; + + uint64_t parts_count; + readText(parts_count, in); + assertChar('\n', in); + + for (uint64_t i = 0; i < parts_count; ++i) + { + std::string partition_id; + std::string part_name; + + readText(partition_id, in); + assertChar('\t', in); + readText(part_name, in); + assertChar('\n', in); + + parts_in_quorum[partition_id] = part_name; + } + return parts_in_quorum; + } + + PartitonIdToPartName read_v3(ReadBufferFromString & in) + { + PartitonIdToPartName parts_in_quorum; + + std::string partition_name; + + readText(partition_name, in); + + auto partition_info = MergeTreePartInfo::fromPartName(partition_name, format_version); + parts_in_quorum[partition_info.partition_id] = partition_name; + + return parts_in_quorum; + } +}; + +} \ No newline at end of file diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 8135304b2e1..bf52d84fa93 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include @@ -2617,8 +2618,10 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) Coordination::Stat added_parts_stat; String old_added_parts = zookeeper->get(quorum_last_part_path, &added_parts_stat); + + ReplicatedMergeTreeQuorumWriter writer(old_added_parts, data.format_version); - String new_added_parts = rewriteAddedParts(old_added_parts, part_name); + String new_added_parts = writer.write(part_name); ops.emplace_back(zkutil::makeRemoveRequest(quorum_status_path, stat.version)); ops.emplace_back(zkutil::makeSetRequest(quorum_last_part_path, new_added_parts, added_parts_stat.version)); @@ -2667,76 +2670,6 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) } -using MaxAddedBlocks = std::unordered_map; -using PartsNames = std::unordered_map; - - -MaxAddedBlocks StorageReplicatedMergeTree::getMaxAddedBlocksWithQuorum(const PartsNames & parts) -{ - MaxAddedBlocks max_added_blocks; - for (auto & part : parts) - { - max_added_blocks[part.first] = MergeTreePartInfo::fromPartName(part.second, data.format_version).max_block; - } - return max_added_blocks; -} - - -PartsNames StorageReplicatedMergeTree::getAddedPartsWithQuorum(const String & parts_str) -{ - auto in = ReadBufferFromString(parts_str); - - size_t count; - PartsNames added_parts; - - if (!parts_str.empty()) - { - try - { - in >> "parts_count " >> count >> "\n"; - for (size_t i = 0; i < count; ++i) - { - String partition_id; - String part_name; - in >> partition_id >> "\t" >> part_name >> "\n"; - added_parts[partition_id] = part_name; - } - } - catch (Exception e) - { - String last_added_part; - in >> last_added_part; - auto partition_info = MergeTreePartInfo::fromPartName(last_added_part, data.format_version); - added_parts[partition_info.partition_id] = last_added_part; - } - } - - return added_parts; -} - - -String StorageReplicatedMergeTree::rewriteAddedParts(const String & old_added_parts_str, const String & new_part_name) -{ - auto added_parts = getAddedPartsWithQuorum(old_added_parts_str); - - auto new_part_info = MergeTreePartInfo::fromPartName(new_part_name, data.format_version); - - added_parts[new_part_info.partition_id] = new_part_name; - - String new_parts; - auto out = WriteBufferFromString(new_parts); - - out << "parts_count " << added_parts.size() << "\n"; - - for (auto & added_part : added_parts) - { - out << added_part.first << "\t" << added_part.second << "\n"; - } - - return new_parts; -} - - bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & replica_path, bool to_detached, size_t quorum) { if (auto part = data.getPartIfExists(part_name, {MergeTreeDataPart::State::Outdated, MergeTreeDataPart::State::Deleting})) @@ -2919,7 +2852,7 @@ BlockInputStreams StorageReplicatedMergeTree::read( * 2. Do not read parts that have not yet been written to the quorum of the replicas. * For this you have to synchronously go to ZooKeeper. */ - MaxAddedBlocks max_added_blocks_with_quorum; + PartitionIdToMaxBlock max_added_blocks_with_quorum; if (settings.select_sequential_consistency) { auto zookeeper = getZooKeeper(); @@ -2929,13 +2862,15 @@ BlockInputStreams StorageReplicatedMergeTree::read( if (!added_parts_str.empty()) { - auto added_parts = getAddedPartsWithQuorum(added_parts_str); + ReplicatedMergeTreeQuorumWriter writer(added_parts_str, data.format_version); + auto added_parts = writer.readParts(); + for (auto & added_part : added_parts) if (!data.getActiveContainingPart(added_part.second)) throw Exception("Replica doesn't have part " + added_part.second + " which was successfully written to quorum of other replicas." " Send query to another replica or disable 'select_sequential_consistency' setting.", ErrorCodes::REPLICA_IS_NOT_IN_QUORUM); - max_added_blocks_with_quorum = getMaxAddedBlocksWithQuorum(added_parts); + max_added_blocks_with_quorum = writer.readBlocks(); } } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index acec796c872..593440b44fd 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -210,6 +210,8 @@ private: using LogEntry = ReplicatedMergeTreeLogEntry; using LogEntryPtr = LogEntry::Ptr; + using PartitionIdToMaxBlock = std::unordered_map; + Context & context; zkutil::ZooKeeperPtr current_zookeeper; /// Use only the methods below. @@ -403,20 +405,6 @@ private: */ bool queueTask(); - using MaxAddedBlocks = std::unordered_map; - using PartsNames = std::unordered_map; - - /// Return map with partition_id and max added block in this partition. - /// This is used to filter the added blocks in read(). - MaxAddedBlocks getMaxAddedBlocksWithQuorum(const PartsNames & parts); - - /// Return map with partition_id and the last added part_name in this partition. - PartsNames getAddedPartsWithQuorum(const String & parts_str); - - /// Rewrite data with inserted partiton with quorum. - /// This is added new partition name in data for ZooKeeper. - String rewriteAddedParts(const String & old_added_parts_str, const String & new_added_part_name); - /// Postcondition: /// either leader_election is fully initialized (node in ZK is created and the watching thread is launched) /// or an exception is thrown and leader_election is destroyed. From 4dee095b9d8259613919e6811a83433434bcb35c Mon Sep 17 00:00:00 2001 From: Vadim Date: Mon, 17 Sep 2018 15:43:14 +0300 Subject: [PATCH 13/79] Update MergeTreeDataSelectExecutor.cpp --- dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 926233dbf97..285225b88fd 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -271,8 +271,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( part->minmax_idx.parallelogram, data.minmax_idx_column_types)) continue; - if (!max_blocks_number_to_read.empty() && max_blocks_number_to_read.find(part->info.partition_id) != max_blocks_number_to_read.end() - && part->info.max_block > max_blocks_number_to_read.at(part->info.partition_id)) + auto blocks_iterator = max_blocks_number_to_read.find(part->info.partition_id); + if (blocks_iterator != max_blocks_number_to_read.end() && part->info.max_block > blocks_iterator->second) continue; parts.push_back(part); From df9a01585a9d38b032e95f2bcbde2f2ee465ab7c Mon Sep 17 00:00:00 2001 From: Vadim Date: Mon, 17 Sep 2018 15:51:42 +0300 Subject: [PATCH 14/79] Update ReplicatedMergeTreeQuorumWriter.h --- .../src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h index 85ed1cd39b7..b330cc54738 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h @@ -29,6 +29,7 @@ struct ReplicatedMergeTreeQuorumWriter read(old_added_parts); } + /// Write new parts in string with added parts. std::string write(const std::string & part_name) { WriteBufferFromOwnString out; @@ -72,6 +73,7 @@ struct ReplicatedMergeTreeQuorumWriter added_parts = read_v3(in); } + /// Read blocks when node in ZooKeeper suppors multiple partitions. PartitonIdToPartName read_v2(ReadBufferFromString & in) { PartitonIdToPartName parts_in_quorum; @@ -95,6 +97,7 @@ struct ReplicatedMergeTreeQuorumWriter return parts_in_quorum; } + /// Read added bloks when node in ZooKeeper supports only one partition. PartitonIdToPartName read_v3(ReadBufferFromString & in) { PartitonIdToPartName parts_in_quorum; @@ -110,4 +113,4 @@ struct ReplicatedMergeTreeQuorumWriter } }; -} \ No newline at end of file +} From 3b155638a002a761e2c84fcd13910e7e2f9794ec Mon Sep 17 00:00:00 2001 From: Vadim Date: Mon, 17 Sep 2018 15:54:20 +0300 Subject: [PATCH 15/79] Update ReplicatedMergeTreeQuorumWriter.h --- dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h index b330cc54738..da44616b329 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h @@ -1,12 +1,10 @@ #pragma once -#include -#include +#include #include #include #include #include -#include #include #include From 3fabac31f6c9b3594497fd2525399414184ec479 Mon Sep 17 00:00:00 2001 From: Vadim Date: Mon, 17 Sep 2018 16:02:30 +0300 Subject: [PATCH 16/79] Update ReplicatedMergeTreeQuorumWriter.h --- .../MergeTree/ReplicatedMergeTreeQuorumWriter.h | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h index da44616b329..ebbe44a9600 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h @@ -27,7 +27,7 @@ struct ReplicatedMergeTreeQuorumWriter read(old_added_parts); } - /// Write new parts in string with added parts. + /// Write new parts in string with added parts. std::string write(const std::string & part_name) { WriteBufferFromOwnString out; @@ -64,14 +64,12 @@ struct ReplicatedMergeTreeQuorumWriter { ReadBufferFromString in(str); if (checkString("parts_count ", in)) - { added_parts = read_v2(in); - } else added_parts = read_v3(in); } - /// Read blocks when node in ZooKeeper suppors multiple partitions. + /// Read blocks when node in ZooKeeper suppors multiple partitions. PartitonIdToPartName read_v2(ReadBufferFromString & in) { PartitonIdToPartName parts_in_quorum; @@ -95,7 +93,7 @@ struct ReplicatedMergeTreeQuorumWriter return parts_in_quorum; } - /// Read added bloks when node in ZooKeeper supports only one partition. + /// Read added bloks when node in ZooKeeper supports only one partition. PartitonIdToPartName read_v3(ReadBufferFromString & in) { PartitonIdToPartName parts_in_quorum; @@ -105,7 +103,7 @@ struct ReplicatedMergeTreeQuorumWriter readText(partition_name, in); auto partition_info = MergeTreePartInfo::fromPartName(partition_name, format_version); - parts_in_quorum[partition_info.partition_id] = partition_name; + parts_in_quorum[partition_info.partition_id] = partition_name; return parts_in_quorum; } From e3add7f0c761b4f829a4b348b9c61f93003c86f0 Mon Sep 17 00:00:00 2001 From: Vadim Date: Mon, 17 Sep 2018 16:05:20 +0300 Subject: [PATCH 17/79] Update ReplicatedMergeTreeQuorumWriter.h --- .../ReplicatedMergeTreeQuorumWriter.h | 144 +++++++++--------- 1 file changed, 72 insertions(+), 72 deletions(-) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h index ebbe44a9600..d068d286126 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h @@ -14,99 +14,99 @@ namespace DB struct ReplicatedMergeTreeQuorumWriter { - using PartitionIdToMaxBlock = std::unordered_map; - using PartitonIdToPartName= std::unordered_map; + using PartitionIdToMaxBlock = std::unordered_map; + using PartitonIdToPartName= std::unordered_map; - PartitonIdToPartName added_parts; + PartitonIdToPartName added_parts; - MergeTreeDataFormatVersion format_version; + MergeTreeDataFormatVersion format_version; - ReplicatedMergeTreeQuorumWriter(const std::string & old_added_parts, MergeTreeDataFormatVersion format_version_) - : format_version(format_version_) - { - read(old_added_parts); - } + ReplicatedMergeTreeQuorumWriter(const std::string & old_added_parts, MergeTreeDataFormatVersion format_version_) + : format_version(format_version_) + { + read(old_added_parts); + } - /// Write new parts in string with added parts. - std::string write(const std::string & part_name) - { - WriteBufferFromOwnString out; + /// Write new parts in string with added parts. + std::string write(const std::string & part_name) + { + WriteBufferFromOwnString out; - auto partition_info = MergeTreePartInfo::fromPartName(part_name, format_version); - added_parts[partition_info.partition_id] = part_name; + auto partition_info = MergeTreePartInfo::fromPartName(part_name, format_version); + added_parts[partition_info.partition_id] = part_name; - out << "parts_count " << added_parts.size() << '\n'; + out << "parts_count " << added_parts.size() << '\n'; - for (const auto & part : added_parts) - out << part.first << '\t' << part.second << '\n'; + for (const auto & part : added_parts) + out << part.first << '\t' << part.second << '\n'; - return out.str(); - } + return out.str(); + } - PartitonIdToPartName readParts() - { - return added_parts; - } + PartitonIdToPartName readParts() + { + return added_parts; + } - PartitionIdToMaxBlock readBlocks() - { - PartitionIdToMaxBlock max_added_blocks; + PartitionIdToMaxBlock readBlocks() + { + PartitionIdToMaxBlock max_added_blocks; - for (const auto & part : added_parts) - { - auto partition_info = MergeTreePartInfo::fromPartName(part.second, format_version); - max_added_blocks[part.first] = partition_info.max_block; - } - return max_added_blocks; - } + for (const auto & part : added_parts) + { + auto partition_info = MergeTreePartInfo::fromPartName(part.second, format_version); + max_added_blocks[part.first] = partition_info.max_block; + } + return max_added_blocks; + } - void read(const std::string & str) - { - ReadBufferFromString in(str); - if (checkString("parts_count ", in)) - added_parts = read_v2(in); - else - added_parts = read_v3(in); - } + void read(const std::string & str) + { + ReadBufferFromString in(str); + if (checkString("parts_count ", in)) + added_parts = read_v2(in); + else + added_parts = read_v3(in); + } - /// Read blocks when node in ZooKeeper suppors multiple partitions. - PartitonIdToPartName read_v2(ReadBufferFromString & in) - { - PartitonIdToPartName parts_in_quorum; + /// Read blocks when node in ZooKeeper suppors multiple partitions. + PartitonIdToPartName read_v2(ReadBufferFromString & in) + { + PartitonIdToPartName parts_in_quorum; - uint64_t parts_count; - readText(parts_count, in); - assertChar('\n', in); - - for (uint64_t i = 0; i < parts_count; ++i) - { - std::string partition_id; - std::string part_name; + uint64_t parts_count; + readText(parts_count, in); + assertChar('\n', in); - readText(partition_id, in); - assertChar('\t', in); - readText(part_name, in); - assertChar('\n', in); + for (uint64_t i = 0; i < parts_count; ++i) + { + std::string partition_id; + std::string part_name; - parts_in_quorum[partition_id] = part_name; - } - return parts_in_quorum; - } + readText(partition_id, in); + assertChar('\t', in); + readText(part_name, in); + assertChar('\n', in); - /// Read added bloks when node in ZooKeeper supports only one partition. - PartitonIdToPartName read_v3(ReadBufferFromString & in) - { - PartitonIdToPartName parts_in_quorum; + parts_in_quorum[partition_id] = part_name; + } + return parts_in_quorum; + } - std::string partition_name; + /// Read added bloks when node in ZooKeeper supports only one partition. + PartitonIdToPartName read_v3(ReadBufferFromString & in) + { + PartitonIdToPartName parts_in_quorum; - readText(partition_name, in); + std::string partition_name; - auto partition_info = MergeTreePartInfo::fromPartName(partition_name, format_version); - parts_in_quorum[partition_info.partition_id] = partition_name; + readText(partition_name, in); - return parts_in_quorum; - } + auto partition_info = MergeTreePartInfo::fromPartName(partition_name, format_version); + parts_in_quorum[partition_info.partition_id] = partition_name; + + return parts_in_quorum; + } }; } From 96b117cb00b2b18828688d57602ce4159b04382e Mon Sep 17 00:00:00 2001 From: VadimPE Date: Wed, 19 Sep 2018 13:23:52 +0300 Subject: [PATCH 18/79] Revert "Update ReplicatedMergeTreeQuorumWriter.h" This reverts commit e3add7f0c761b4f829a4b348b9c61f93003c86f0. --- .../ReplicatedMergeTreeQuorumWriter.h | 144 +++++++++--------- 1 file changed, 72 insertions(+), 72 deletions(-) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h index d068d286126..ebbe44a9600 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h @@ -14,99 +14,99 @@ namespace DB struct ReplicatedMergeTreeQuorumWriter { - using PartitionIdToMaxBlock = std::unordered_map; - using PartitonIdToPartName= std::unordered_map; + using PartitionIdToMaxBlock = std::unordered_map; + using PartitonIdToPartName= std::unordered_map; - PartitonIdToPartName added_parts; + PartitonIdToPartName added_parts; - MergeTreeDataFormatVersion format_version; + MergeTreeDataFormatVersion format_version; - ReplicatedMergeTreeQuorumWriter(const std::string & old_added_parts, MergeTreeDataFormatVersion format_version_) - : format_version(format_version_) - { - read(old_added_parts); - } + ReplicatedMergeTreeQuorumWriter(const std::string & old_added_parts, MergeTreeDataFormatVersion format_version_) + : format_version(format_version_) + { + read(old_added_parts); + } - /// Write new parts in string with added parts. - std::string write(const std::string & part_name) - { - WriteBufferFromOwnString out; + /// Write new parts in string with added parts. + std::string write(const std::string & part_name) + { + WriteBufferFromOwnString out; - auto partition_info = MergeTreePartInfo::fromPartName(part_name, format_version); - added_parts[partition_info.partition_id] = part_name; + auto partition_info = MergeTreePartInfo::fromPartName(part_name, format_version); + added_parts[partition_info.partition_id] = part_name; - out << "parts_count " << added_parts.size() << '\n'; + out << "parts_count " << added_parts.size() << '\n'; - for (const auto & part : added_parts) - out << part.first << '\t' << part.second << '\n'; + for (const auto & part : added_parts) + out << part.first << '\t' << part.second << '\n'; - return out.str(); - } + return out.str(); + } - PartitonIdToPartName readParts() - { - return added_parts; - } + PartitonIdToPartName readParts() + { + return added_parts; + } - PartitionIdToMaxBlock readBlocks() - { - PartitionIdToMaxBlock max_added_blocks; + PartitionIdToMaxBlock readBlocks() + { + PartitionIdToMaxBlock max_added_blocks; - for (const auto & part : added_parts) - { - auto partition_info = MergeTreePartInfo::fromPartName(part.second, format_version); - max_added_blocks[part.first] = partition_info.max_block; - } - return max_added_blocks; - } + for (const auto & part : added_parts) + { + auto partition_info = MergeTreePartInfo::fromPartName(part.second, format_version); + max_added_blocks[part.first] = partition_info.max_block; + } + return max_added_blocks; + } - void read(const std::string & str) - { - ReadBufferFromString in(str); - if (checkString("parts_count ", in)) - added_parts = read_v2(in); - else - added_parts = read_v3(in); - } + void read(const std::string & str) + { + ReadBufferFromString in(str); + if (checkString("parts_count ", in)) + added_parts = read_v2(in); + else + added_parts = read_v3(in); + } - /// Read blocks when node in ZooKeeper suppors multiple partitions. - PartitonIdToPartName read_v2(ReadBufferFromString & in) - { - PartitonIdToPartName parts_in_quorum; + /// Read blocks when node in ZooKeeper suppors multiple partitions. + PartitonIdToPartName read_v2(ReadBufferFromString & in) + { + PartitonIdToPartName parts_in_quorum; - uint64_t parts_count; - readText(parts_count, in); - assertChar('\n', in); + uint64_t parts_count; + readText(parts_count, in); + assertChar('\n', in); + + for (uint64_t i = 0; i < parts_count; ++i) + { + std::string partition_id; + std::string part_name; - for (uint64_t i = 0; i < parts_count; ++i) - { - std::string partition_id; - std::string part_name; + readText(partition_id, in); + assertChar('\t', in); + readText(part_name, in); + assertChar('\n', in); - readText(partition_id, in); - assertChar('\t', in); - readText(part_name, in); - assertChar('\n', in); + parts_in_quorum[partition_id] = part_name; + } + return parts_in_quorum; + } - parts_in_quorum[partition_id] = part_name; - } - return parts_in_quorum; - } + /// Read added bloks when node in ZooKeeper supports only one partition. + PartitonIdToPartName read_v3(ReadBufferFromString & in) + { + PartitonIdToPartName parts_in_quorum; - /// Read added bloks when node in ZooKeeper supports only one partition. - PartitonIdToPartName read_v3(ReadBufferFromString & in) - { - PartitonIdToPartName parts_in_quorum; + std::string partition_name; - std::string partition_name; + readText(partition_name, in); - readText(partition_name, in); + auto partition_info = MergeTreePartInfo::fromPartName(partition_name, format_version); + parts_in_quorum[partition_info.partition_id] = partition_name; - auto partition_info = MergeTreePartInfo::fromPartName(partition_name, format_version); - parts_in_quorum[partition_info.partition_id] = partition_name; - - return parts_in_quorum; - } + return parts_in_quorum; + } }; } From 17ffa8c31022b60c311d2701deba05d140199106 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Wed, 19 Sep 2018 14:08:04 +0300 Subject: [PATCH 19/79] fix class for added parts --- .../ReplicatedMergeTreeQuorumAddedParts.h | 127 ++++++++++++++++++ .../Storages/StorageReplicatedMergeTree.cpp | 17 ++- .../src/Storages/StorageReplicatedMergeTree.h | 2 - 3 files changed, 137 insertions(+), 9 deletions(-) create mode 100644 dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h new file mode 100644 index 00000000000..664990fee94 --- /dev/null +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h @@ -0,0 +1,127 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +struct ReplicatedMergeTreeQuorumAddedParts +{ + using PartitionIdToMaxBlock = std::unordered_map; + using PartitonIdToPartName= std::unordered_map; + + PartitonIdToPartName added_parts; + + MergeTreeDataFormatVersion format_version; + + ReplicatedMergeTreeQuorumAddedParts(const std::string & added_parts_str, MergeTreeDataFormatVersion format_version_) + : format_version(format_version_) + { + fromString(added_parts_str); + } + + /// Write new parts in buffer with added parts. + void write(WriteBufferFromOwnString & out) + { + out << "version: " << 2 << '\n'; + out << "parts count: " << added_parts.size() << '\n'; + + for (const auto & part : added_parts) + out << part.first << '\t' << part.second << '\n'; + } + + PartitionIdToMaxBlock getMaxInsertedBlocks() + { + PartitionIdToMaxBlock max_added_blocks; + + for (const auto & part : added_parts) + { + auto partition_info = MergeTreePartInfo::fromPartName(part.second, format_version); + max_added_blocks[part.first] = partition_info.max_block; + } + + return max_added_blocks; + } + + void read(ReadBufferFromString & in) + { + if (checkString("version: ", in)) + { + size_t version; + + readText(version, in); + assertChar('\n', in); + + if (version == 2) + added_parts = read_v2(in); + } + else + added_parts = read_v1(in); + } + + /// Read added bloks when node in ZooKeeper supports only one partition. + PartitonIdToPartName read_v1(ReadBufferFromString & in) + { + PartitonIdToPartName parts_in_quorum; + + std::string partition_name; + + readText(partition_name, in); + + auto partition_info = MergeTreePartInfo::fromPartName(partition_name, format_version); + parts_in_quorum[partition_info.partition_id] = partition_name; + + return parts_in_quorum; + } + + /// Read blocks when node in ZooKeeper suppors multiple partitions. + PartitonIdToPartName read_v2(ReadBufferFromString & in) + { + assertString("parts count: ", in); + + PartitonIdToPartName parts_in_quorum; + + uint64_t parts_count; + readText(parts_count, in); + assertChar('\n', in); + + for (uint64_t i = 0; i < parts_count; ++i) + { + std::string partition_id; + std::string part_name; + + readText(partition_id, in); + assertChar('\t', in); + readText(part_name, in); + assertChar('\n', in); + + parts_in_quorum[partition_id] = part_name; + } + return parts_in_quorum; + } + + void fromString(const std::string & str) + { + if (str.empty()) + return; + ReadBufferFromString in(str); + read(in); + } + + std::string toString() + { + WriteBufferFromOwnString out; + write(out); + return out.str(); + } + +}; + +} diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index e4e1c79465e..7123c79fb02 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include @@ -2669,9 +2669,12 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) Coordination::Stat added_parts_stat; String old_added_parts = zookeeper->get(quorum_last_part_path, &added_parts_stat); - ReplicatedMergeTreeQuorumWriter writer(old_added_parts, data.format_version); + ReplicatedMergeTreeQuorumAddedParts parts_with_quorum(old_added_parts, data.format_version); + + auto partition_info = MergeTreePartInfo::fromPartName(part_name, data.format_version); + parts_with_quorum.added_parts[partition_info.partition_id] = partition_info.max_block; - String new_added_parts = writer.write(part_name); + String new_added_parts = parts_with_quorum.toString(); ops.emplace_back(zkutil::makeRemoveRequest(quorum_status_path, stat.version)); ops.emplace_back(zkutil::makeSetRequest(quorum_last_part_path, new_added_parts, added_parts_stat.version)); @@ -2949,7 +2952,7 @@ BlockInputStreams StorageReplicatedMergeTree::read( * 2. Do not read parts that have not yet been written to the quorum of the replicas. * For this you have to synchronously go to ZooKeeper. */ - PartitionIdToMaxBlock max_added_blocks_with_quorum; + DB::ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock max_added_blocks_with_quorum; if (settings.select_sequential_consistency) { auto zookeeper = getZooKeeper(); @@ -2959,15 +2962,15 @@ BlockInputStreams StorageReplicatedMergeTree::read( if (!added_parts_str.empty()) { - ReplicatedMergeTreeQuorumWriter writer(added_parts_str, data.format_version); - auto added_parts = writer.readParts(); + ReplicatedMergeTreeQuorumAddedParts part_with_quorum(added_parts_str, data.format_version); + auto added_parts = part_with_quorum.added_parts; for (auto & added_part : added_parts) if (!data.getActiveContainingPart(added_part.second)) throw Exception("Replica doesn't have part " + added_part.second + " which was successfully written to quorum of other replicas." " Send query to another replica or disable 'select_sequential_consistency' setting.", ErrorCodes::REPLICA_IS_NOT_IN_QUORUM); - max_added_blocks_with_quorum = writer.readBlocks(); + max_added_blocks_with_quorum = part_with_quorum.getMaxInsertedBlocks(); } } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 45e26863526..779e202fdff 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -212,8 +212,6 @@ private: using LogEntry = ReplicatedMergeTreeLogEntry; using LogEntryPtr = LogEntry::Ptr; - using PartitionIdToMaxBlock = std::unordered_map; - Context & context; zkutil::ZooKeeperPtr current_zookeeper; /// Use only the methods below. From f05d93897b5777b23178aa193166079636b8607e Mon Sep 17 00:00:00 2001 From: VadimPE Date: Wed, 19 Sep 2018 14:10:56 +0300 Subject: [PATCH 20/79] Del old class --- .../ReplicatedMergeTreeQuorumWriter.h | 112 ------------------ 1 file changed, 112 deletions(-) delete mode 100644 dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h deleted file mode 100644 index ebbe44a9600..00000000000 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumWriter.h +++ /dev/null @@ -1,112 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include - -#include - -namespace DB -{ - -struct ReplicatedMergeTreeQuorumWriter -{ - using PartitionIdToMaxBlock = std::unordered_map; - using PartitonIdToPartName= std::unordered_map; - - PartitonIdToPartName added_parts; - - MergeTreeDataFormatVersion format_version; - - ReplicatedMergeTreeQuorumWriter(const std::string & old_added_parts, MergeTreeDataFormatVersion format_version_) - : format_version(format_version_) - { - read(old_added_parts); - } - - /// Write new parts in string with added parts. - std::string write(const std::string & part_name) - { - WriteBufferFromOwnString out; - - auto partition_info = MergeTreePartInfo::fromPartName(part_name, format_version); - added_parts[partition_info.partition_id] = part_name; - - out << "parts_count " << added_parts.size() << '\n'; - - for (const auto & part : added_parts) - out << part.first << '\t' << part.second << '\n'; - - return out.str(); - } - - PartitonIdToPartName readParts() - { - return added_parts; - } - - PartitionIdToMaxBlock readBlocks() - { - PartitionIdToMaxBlock max_added_blocks; - - for (const auto & part : added_parts) - { - auto partition_info = MergeTreePartInfo::fromPartName(part.second, format_version); - max_added_blocks[part.first] = partition_info.max_block; - } - return max_added_blocks; - } - - void read(const std::string & str) - { - ReadBufferFromString in(str); - if (checkString("parts_count ", in)) - added_parts = read_v2(in); - else - added_parts = read_v3(in); - } - - /// Read blocks when node in ZooKeeper suppors multiple partitions. - PartitonIdToPartName read_v2(ReadBufferFromString & in) - { - PartitonIdToPartName parts_in_quorum; - - uint64_t parts_count; - readText(parts_count, in); - assertChar('\n', in); - - for (uint64_t i = 0; i < parts_count; ++i) - { - std::string partition_id; - std::string part_name; - - readText(partition_id, in); - assertChar('\t', in); - readText(part_name, in); - assertChar('\n', in); - - parts_in_quorum[partition_id] = part_name; - } - return parts_in_quorum; - } - - /// Read added bloks when node in ZooKeeper supports only one partition. - PartitonIdToPartName read_v3(ReadBufferFromString & in) - { - PartitonIdToPartName parts_in_quorum; - - std::string partition_name; - - readText(partition_name, in); - - auto partition_info = MergeTreePartInfo::fromPartName(partition_name, format_version); - parts_in_quorum[partition_info.partition_id] = partition_name; - - return parts_in_quorum; - } -}; - -} From 66b4a606d5e7a16f22c3aa9344b92c6e644d30a5 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Wed, 19 Sep 2018 17:34:41 +0300 Subject: [PATCH 21/79] fix bug with add new partition in quorum --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 15 +++++++++++ dbms/src/Storages/MergeTree/MergeTreeData.h | 4 +++ .../ReplicatedMergeTreeQuorumAddedParts.h | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 26 ++++++++++++++++--- 4 files changed, 43 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index f140fbaa5b5..b48af42ac9f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1785,6 +1785,21 @@ size_t MergeTreeData::getMaxPartsCountForPartition() const } +using PartitionIdToMaxBlock = std::unordered_map; + +PartitionIdToMaxBlock MergeTreeData::getMaxBlocksForPartition() const +{ + std::lock_guard lock(data_parts_mutex); + + PartitionIdToMaxBlock max_blocks; + + for (const auto & part : getDataPartsStateRange(DataPartState::Committed)) + max_blocks[part->info.partition_id] = part->info.max_block; + + return max_blocks; +} + + std::optional MergeTreeData::getMinPartDataVersion() const { std::lock_guard lock(data_parts_mutex); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 09aff9dae1d..fed04b69954 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -388,6 +388,10 @@ public: size_t getMaxPartsCountForPartition() const; + using PartitionIdToMaxBlock = std::unordered_map; + + PartitionIdToMaxBlock getMaxBlocksForPartition() const; + /// Get min value of part->info.getDataVersion() for all active parts. /// Makes sense only for ordinary MergeTree engines because for them block numbering doesn't depend on partition. std::optional getMinPartDataVersion() const; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h index 664990fee94..029aec8db8a 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h @@ -15,7 +15,7 @@ namespace DB struct ReplicatedMergeTreeQuorumAddedParts { using PartitionIdToMaxBlock = std::unordered_map; - using PartitonIdToPartName= std::unordered_map; + using PartitonIdToPartName = std::unordered_map; PartitonIdToPartName added_parts; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 7123c79fb02..c9320c19fc9 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -50,6 +50,8 @@ #include #include +#include + namespace ProfileEvents { @@ -2952,11 +2954,28 @@ BlockInputStreams StorageReplicatedMergeTree::read( * 2. Do not read parts that have not yet been written to the quorum of the replicas. * For this you have to synchronously go to ZooKeeper. */ - DB::ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock max_added_blocks_with_quorum; + DB::ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock max_added_blocks; if (settings.select_sequential_consistency) { + max_added_blocks = data.getMaxBlocksForPartition(); + auto zookeeper = getZooKeeper(); + const String quorum_last_part_path = zookeeper_path + "/quorum/last_part"; + + String value; + Coordination::Stat stat; + + if (zookeeper->tryGet(quorum_last_part_path, value, &stat)) + { + ReplicatedMergeTreeQuorumEntry quorum_entry; + quorum_entry.fromString(value); + + auto partition_info = MergeTreePartInfo::fromPartName(quorum_entry.part_name, data.format_version); + + max_added_blocks[partition_info.partition_id] = partition_info.max_block - 1; + } + String added_parts_str; zookeeper->tryGet(zookeeper_path + "/quorum/last_part", added_parts_str); @@ -2970,11 +2989,12 @@ BlockInputStreams StorageReplicatedMergeTree::read( throw Exception("Replica doesn't have part " + added_part.second + " which was successfully written to quorum of other replicas." " Send query to another replica or disable 'select_sequential_consistency' setting.", ErrorCodes::REPLICA_IS_NOT_IN_QUORUM); - max_added_blocks_with_quorum = part_with_quorum.getMaxInsertedBlocks(); + for (const auto & max_block : part_with_quorum.getMaxInsertedBlocks()) + max_added_blocks[max_block.first] = max_block.second; } } - return reader.read(column_names, query_info, context, max_block_size, num_streams, max_added_blocks_with_quorum); + return reader.read(column_names, query_info, context, max_block_size, num_streams, max_added_blocks); } From 2396b445193bcaaa28b8f42eea0cf5bf8dda90e0 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Wed, 19 Sep 2018 17:44:30 +0300 Subject: [PATCH 22/79] fix --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index c9320c19fc9..4a10d86b27e 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2961,12 +2961,12 @@ BlockInputStreams StorageReplicatedMergeTree::read( auto zookeeper = getZooKeeper(); - const String quorum_last_part_path = zookeeper_path + "/quorum/last_part"; + const String quorum_status_path = zookeeper_path + "/quorum/status"; String value; Coordination::Stat stat; - if (zookeeper->tryGet(quorum_last_part_path, value, &stat)) + if (zookeeper->tryGet(quorum_status_path, value, &stat)) { ReplicatedMergeTreeQuorumEntry quorum_entry; quorum_entry.fromString(value); From 89d408224b52e24499f9df2693a54c29e630d5b4 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Wed, 19 Sep 2018 18:07:20 +0300 Subject: [PATCH 23/79] fix bug with part_name --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 4a10d86b27e..d7f2d7e45b9 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2674,7 +2674,7 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) ReplicatedMergeTreeQuorumAddedParts parts_with_quorum(old_added_parts, data.format_version); auto partition_info = MergeTreePartInfo::fromPartName(part_name, data.format_version); - parts_with_quorum.added_parts[partition_info.partition_id] = partition_info.max_block; + parts_with_quorum.added_parts[partition_info.partition_id] = part_name; String new_added_parts = parts_with_quorum.toString(); From 33545a41b6cf8d06a5611cda98c38ca7615afb3a Mon Sep 17 00:00:00 2001 From: VadimPE Date: Fri, 21 Sep 2018 18:07:43 +0300 Subject: [PATCH 24/79] fix style. And del getMaxBlock...() --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 15 --------------- dbms/src/Storages/MergeTree/MergeTreeData.h | 4 ---- .../ReplicatedMergeTreeQuorumAddedParts.h | 16 ++++++++-------- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 7 +++++-- 4 files changed, 13 insertions(+), 29 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index b48af42ac9f..f140fbaa5b5 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1785,21 +1785,6 @@ size_t MergeTreeData::getMaxPartsCountForPartition() const } -using PartitionIdToMaxBlock = std::unordered_map; - -PartitionIdToMaxBlock MergeTreeData::getMaxBlocksForPartition() const -{ - std::lock_guard lock(data_parts_mutex); - - PartitionIdToMaxBlock max_blocks; - - for (const auto & part : getDataPartsStateRange(DataPartState::Committed)) - max_blocks[part->info.partition_id] = part->info.max_block; - - return max_blocks; -} - - std::optional MergeTreeData::getMinPartDataVersion() const { std::lock_guard lock(data_parts_mutex); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index fed04b69954..09aff9dae1d 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -388,10 +388,6 @@ public: size_t getMaxPartsCountForPartition() const; - using PartitionIdToMaxBlock = std::unordered_map; - - PartitionIdToMaxBlock getMaxBlocksForPartition() const; - /// Get min value of part->info.getDataVersion() for all active parts. /// Makes sense only for ordinary MergeTree engines because for them block numbering doesn't depend on partition. std::optional getMinPartDataVersion() const; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h index 029aec8db8a..33f181aa17e 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h @@ -15,9 +15,9 @@ namespace DB struct ReplicatedMergeTreeQuorumAddedParts { using PartitionIdToMaxBlock = std::unordered_map; - using PartitonIdToPartName = std::unordered_map; + using PartitionIdToPartName = std::unordered_map; - PartitonIdToPartName added_parts; + PartitionIdToPartName added_parts; MergeTreeDataFormatVersion format_version; @@ -28,7 +28,7 @@ struct ReplicatedMergeTreeQuorumAddedParts } /// Write new parts in buffer with added parts. - void write(WriteBufferFromOwnString & out) + void write(WriteBuffer & out) { out << "version: " << 2 << '\n'; out << "parts count: " << added_parts.size() << '\n'; @@ -50,7 +50,7 @@ struct ReplicatedMergeTreeQuorumAddedParts return max_added_blocks; } - void read(ReadBufferFromString & in) + void read(ReadBuffer & in) { if (checkString("version: ", in)) { @@ -67,9 +67,9 @@ struct ReplicatedMergeTreeQuorumAddedParts } /// Read added bloks when node in ZooKeeper supports only one partition. - PartitonIdToPartName read_v1(ReadBufferFromString & in) + PartitionIdToPartName read_v1(ReadBuffer & in) { - PartitonIdToPartName parts_in_quorum; + PartitionIdToPartName parts_in_quorum; std::string partition_name; @@ -82,11 +82,11 @@ struct ReplicatedMergeTreeQuorumAddedParts } /// Read blocks when node in ZooKeeper suppors multiple partitions. - PartitonIdToPartName read_v2(ReadBufferFromString & in) + PartitionIdToPartName read_v2(ReadBuffer & in) { assertString("parts count: ", in); - PartitonIdToPartName parts_in_quorum; + PartitionIdToPartName parts_in_quorum; uint64_t parts_count; readText(parts_count, in); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index d7f2d7e45b9..7ea6843f14a 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2954,10 +2954,13 @@ BlockInputStreams StorageReplicatedMergeTree::read( * 2. Do not read parts that have not yet been written to the quorum of the replicas. * For this you have to synchronously go to ZooKeeper. */ - DB::ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock max_added_blocks; + ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock max_added_blocks; if (settings.select_sequential_consistency) { - max_added_blocks = data.getMaxBlocksForPartition(); + for (const auto & data_part : data.getDataParts()) + { + max_added_blocks[data_part->info.partition_id] = data_part->info.max_block; + } auto zookeeper = getZooKeeper(); From 17aa0356e524f7b15c9ce374c20feeb683accd75 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Fri, 21 Sep 2018 18:15:35 +0300 Subject: [PATCH 25/79] del include --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 7ea6843f14a..64a05c41d4e 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -50,8 +50,6 @@ #include #include -#include - namespace ProfileEvents { From 8c413e3f1f3d199c98ac34a4e8ccea0f8aca5896 Mon Sep 17 00:00:00 2001 From: Vadim Date: Fri, 21 Sep 2018 18:20:31 +0300 Subject: [PATCH 26/79] Update ReplicatedMergeTreeQuorumAddedParts.h --- .../ReplicatedMergeTreeQuorumAddedParts.h | 168 +++++++++--------- 1 file changed, 84 insertions(+), 84 deletions(-) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h index 33f181aa17e..0e971b42d0d 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h @@ -14,113 +14,113 @@ namespace DB struct ReplicatedMergeTreeQuorumAddedParts { - using PartitionIdToMaxBlock = std::unordered_map; - using PartitionIdToPartName = std::unordered_map; + using PartitionIdToMaxBlock = std::unordered_map; + using PartitionIdToPartName = std::unordered_map; - PartitionIdToPartName added_parts; + PartitionIdToPartName added_parts; - MergeTreeDataFormatVersion format_version; + MergeTreeDataFormatVersion format_version; - ReplicatedMergeTreeQuorumAddedParts(const std::string & added_parts_str, MergeTreeDataFormatVersion format_version_) - : format_version(format_version_) - { - fromString(added_parts_str); - } + ReplicatedMergeTreeQuorumAddedParts(const std::string & added_parts_str, MergeTreeDataFormatVersion format_version_) + : format_version(format_version_) + { + fromString(added_parts_str); + } - /// Write new parts in buffer with added parts. - void write(WriteBuffer & out) - { - out << "version: " << 2 << '\n'; - out << "parts count: " << added_parts.size() << '\n'; + /// Write new parts in buffer with added parts. + void write(WriteBuffer & out) + { + out << "version: " << 2 << '\n'; + out << "parts count: " << added_parts.size() << '\n'; - for (const auto & part : added_parts) - out << part.first << '\t' << part.second << '\n'; - } + for (const auto & part : added_parts) + out << part.first << '\t' << part.second << '\n'; + } - PartitionIdToMaxBlock getMaxInsertedBlocks() - { - PartitionIdToMaxBlock max_added_blocks; + PartitionIdToMaxBlock getMaxInsertedBlocks() + { + PartitionIdToMaxBlock max_added_blocks; - for (const auto & part : added_parts) - { - auto partition_info = MergeTreePartInfo::fromPartName(part.second, format_version); - max_added_blocks[part.first] = partition_info.max_block; - } - - return max_added_blocks; - } + for (const auto & part : added_parts) + { + auto partition_info = MergeTreePartInfo::fromPartName(part.second, format_version); + max_added_blocks[part.first] = partition_info.max_block; + } - void read(ReadBuffer & in) - { - if (checkString("version: ", in)) - { - size_t version; + return max_added_blocks; + } - readText(version, in); - assertChar('\n', in); + void read(ReadBuffer & in) + { + if (checkString("version: ", in)) + { + size_t version; - if (version == 2) - added_parts = read_v2(in); - } - else - added_parts = read_v1(in); - } + readText(version, in); + assertChar('\n', in); - /// Read added bloks when node in ZooKeeper supports only one partition. - PartitionIdToPartName read_v1(ReadBuffer & in) - { - PartitionIdToPartName parts_in_quorum; + if (version == 2) + added_parts = read_v2(in); + } + else + added_parts = read_v1(in); + } - std::string partition_name; + /// Read added bloks when node in ZooKeeper supports only one partition. + PartitionIdToPartName read_v1(ReadBuffer & in) + { + PartitionIdToPartName parts_in_quorum; - readText(partition_name, in); + std::string partition_name; - auto partition_info = MergeTreePartInfo::fromPartName(partition_name, format_version); + readText(partition_name, in); + + auto partition_info = MergeTreePartInfo::fromPartName(partition_name, format_version); parts_in_quorum[partition_info.partition_id] = partition_name; - return parts_in_quorum; - } + return parts_in_quorum; + } - /// Read blocks when node in ZooKeeper suppors multiple partitions. - PartitionIdToPartName read_v2(ReadBuffer & in) - { - assertString("parts count: ", in); + /// Read blocks when node in ZooKeeper suppors multiple partitions. + PartitionIdToPartName read_v2(ReadBuffer & in) + { + assertString("parts count: ", in); - PartitionIdToPartName parts_in_quorum; + PartitionIdToPartName parts_in_quorum; - uint64_t parts_count; - readText(parts_count, in); - assertChar('\n', in); - - for (uint64_t i = 0; i < parts_count; ++i) - { - std::string partition_id; - std::string part_name; + uint64_t parts_count; + readText(parts_count, in); + assertChar('\n', in); - readText(partition_id, in); - assertChar('\t', in); - readText(part_name, in); - assertChar('\n', in); + for (uint64_t i = 0; i < parts_count; ++i) + { + std::string partition_id; + std::string part_name; - parts_in_quorum[partition_id] = part_name; - } - return parts_in_quorum; - } + readText(partition_id, in); + assertChar('\t', in); + readText(part_name, in); + assertChar('\n', in); - void fromString(const std::string & str) - { - if (str.empty()) - return; - ReadBufferFromString in(str); - read(in); - } + parts_in_quorum[partition_id] = part_name; + } + return parts_in_quorum; + } - std::string toString() - { - WriteBufferFromOwnString out; - write(out); - return out.str(); - } + void fromString(const std::string & str) + { + if (str.empty()) + return; + ReadBufferFromString in(str); + read(in); + } + + std::string toString() + { + WriteBufferFromOwnString out; + write(out); + return out.str(); + } }; From 28102bc3b525932c770d4ef6647af43752d95021 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Mon, 24 Sep 2018 12:53:28 +0300 Subject: [PATCH 27/79] ISSUES-2581 get pointer to max_blocls --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 13 ++++++++----- .../MergeTree/MergeTreeDataSelectExecutor.h | 4 ++-- .../MergeTree/StorageFromMergeTreeDataPart.h | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 2 +- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 5 ++++- 5 files changed, 16 insertions(+), 10 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 285225b88fd..7ed38774a8e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -137,7 +137,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read( const Context & context, const size_t max_block_size, const unsigned num_streams, - const PartitionIdToMaxBlock & max_blocks_number_to_read) const + const PartitionIdToMaxBlock* max_blocks_number_to_read) const { return readFromParts( data.getDataPartsVector(), column_names_to_return, query_info, context, @@ -151,7 +151,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( const Context & context, const size_t max_block_size, const unsigned num_streams, - const PartitionIdToMaxBlock & max_blocks_number_to_read) const + const PartitionIdToMaxBlock* max_blocks_number_to_read) const { size_t part_index = 0; @@ -271,9 +271,12 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( part->minmax_idx.parallelogram, data.minmax_idx_column_types)) continue; - auto blocks_iterator = max_blocks_number_to_read.find(part->info.partition_id); - if (blocks_iterator != max_blocks_number_to_read.end() && part->info.max_block > blocks_iterator->second) - continue; + if (max_blocks_number_to_read) + { + auto blocks_iterator = max_blocks_number_to_read->find(part->info.partition_id); + if (blocks_iterator != max_blocks_number_to_read->end() && part->info.max_block > blocks_iterator->second) + continue; + } parts.push_back(part); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 9da2a95bd9a..d3715960fc2 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -30,7 +30,7 @@ public: const Context & context, size_t max_block_size, unsigned num_streams, - const PartitionIdToMaxBlock & max_blocks_number_to_read) const; + const PartitionIdToMaxBlock* max_blocks_number_to_read = nullptr) const; BlockInputStreams readFromParts( MergeTreeData::DataPartsVector parts, @@ -39,7 +39,7 @@ public: const Context & context, size_t max_block_size, unsigned num_streams, - const PartitionIdToMaxBlock & max_blocks_number_to_read) const; + const PartitionIdToMaxBlock* max_blocks_number_to_read = nullptr) const; private: MergeTreeData & data; diff --git a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 33298aa9158..1447ffb40d8 100644 --- a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -27,7 +27,7 @@ public: unsigned num_streams) override { return MergeTreeDataSelectExecutor(part->storage).readFromParts( - {part}, column_names, query_info, context, max_block_size, num_streams, {}); + {part}, column_names, query_info, context, max_block_size, num_streams); } bool supportsIndexForIn() const override { return true; } diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index a2d2e3da500..4628743ac7f 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -114,7 +114,7 @@ BlockInputStreams StorageMergeTree::read( const size_t max_block_size, const unsigned num_streams) { - return reader.read(column_names, query_info, context, max_block_size, num_streams, {}); + return reader.read(column_names, query_info, context, max_block_size, num_streams); } BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & /*query*/, const Settings & /*settings*/) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 64a05c41d4e..ac7d85aa9f1 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2995,7 +2995,10 @@ BlockInputStreams StorageReplicatedMergeTree::read( } } - return reader.read(column_names, query_info, context, max_block_size, num_streams, max_added_blocks); + if (max_added_blocks.empty()) + return reader.read(column_names, query_info, context, max_block_size, num_streams); + else + return reader.read(column_names, query_info, context, max_block_size, num_streams, &max_added_blocks); } From dd9516a810de265807046db4ce37ae0ed7924511 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Thu, 4 Oct 2018 13:24:51 +0300 Subject: [PATCH 28/79] wip on finish sorting --- .../MergeSortingBlockInputStream.cpp | 123 ++++++++++++++++++ .../MergeSortingBlockInputStream.h | 41 ++++++ dbms/src/Interpreters/sortBlock.cpp | 46 ++----- dbms/src/Interpreters/sortBlock.h | 24 ++++ 4 files changed, 200 insertions(+), 34 deletions(-) diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp index 6122f54630d..28c8c6c2094 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace ProfileEvents @@ -304,4 +305,126 @@ void MergeSortingBlockInputStream::remerge() sum_bytes_in_blocks = new_sum_bytes_in_blocks; } + +FinishMergeSortingBlockInputStream::FinishMergeSortingBlockInputStream( + const BlockInputStreamPtr & input, SortDescription & description_sorted_, + SortDescription & description_to_sort_, + size_t max_merged_block_size_, size_t limit_) + : description_sorted(description_sorted_), description_to_sort(description_to_sort_), + max_merged_block_size(max_merged_block_size_), limit(limit_) +{ + children.push_back(input); + header = children.at(0)->getHeader(); + removeConstantsFromSortDescription(header, description_sorted); + removeConstantsFromSortDescription(header, description_to_sort); +} + +static bool equalKeysAt(const ColumnsWithSortDescriptions & lhs, const ColumnsWithSortDescriptions & rhs, size_t n, size_t m) +{ + + for (auto it = lhs.begin(), jt = rhs.begin(); it != lhs.end(); ++it, ++jt) + { + int res = it->first->compareAt(n, m, *jt->first, it->second.nulls_direction); + if (res != 0) + return false; + } + return true; +} + +Block FinishMergeSortingBlockInputStream::readImpl() +{ + if (limit && total_rows_processed == limit) + return {}; + + Block res; + if (impl) + res = impl->read(); + + /// If res block is empty, we finish sorting previous chunk of blocks. + if (!res) + { + if (end_of_stream) + return {}; + + blocks.clear(); + if (tail_block) + blocks.push_back(std::move(tail_block)); + + Block block; + size_t tail_pos = 0; + while (true) + { + block = children.back()->read(); + + /// End of input stream, but we can`t returns immediatly, we need to merge already read blocks. + /// Check it later, when get end of stream from impl. + if (!block) + { + end_of_stream = true; + break; + } + + // If there were only const columns in sort description, then there is no need to sort. + // Return the blocks as is. + if (description_to_sort.empty()) + return block; + + size_t size = block.rows(); + if (size == 0) + continue; + + auto columns_with_sort_desc = getColumnsWithSortDescription(block, description_sorted); + + removeConstantsFromBlock(block); + + /// May be new block starts with new key. + if (!blocks.empty()) + { + const Block & last_block = blocks.back(); + if (!equalKeysAt(getColumnsWithSortDescription(last_block, description_sorted), columns_with_sort_desc, last_block.rows() - 1, 0)) + break; + } + + IColumn::Permutation perm(size); + for (size_t i = 0; i < size; ++i) + perm[i] = i; + + PartialSortingLess less(columns_with_sort_desc); + + /// We need to save tail of block, because next block may starts with the same key as in tail + /// and we should sort these rows in one chunk. + tail_pos = *std::lower_bound(perm.begin(), perm.end(), size - 1, less); + + if (tail_pos != 0) + break; + + /// If we reach here, that means that current block has all rows with the same key as tail of a previous block. + blocks.push_back(block); + } + + if (block) + { + Block head_block = block.cloneEmpty(); + tail_block = block.cloneEmpty(); + for (size_t i = 0; i < block.columns(); ++i) + { + head_block.getByPosition(i).column = block.getByPosition(i).column->cut(0, tail_pos); + tail_block.getByPosition(i).column = block.getByPosition(i).column->cut(tail_pos, block.rows() - tail_pos); + } + + blocks.push_back(head_block); + } + + impl = std::make_unique(blocks, description_to_sort, max_merged_block_size, limit); + res = impl->read(); + } + + if (res) + enrichBlockWithConstants(res, header); + + total_rows_processed += res.rows(); + + return res; +} + } diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.h b/dbms/src/DataStreams/MergeSortingBlockInputStream.h index 9b7e1aa1139..8a3bdda2750 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.h +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.h @@ -131,4 +131,45 @@ private: bool remerge_is_useful = true; }; + +/** Takes stream already sorted by `x` and finishes sorting it by (`x`, `y`). + * During sorting only blocks with rows equal by `x` saved in RAM. + * */ +class FinishMergeSortingBlockInputStream : public IProfilingBlockInputStream +{ +public: + /// limit - if not 0, allowed to return just first 'limit' rows in sorted order. + FinishMergeSortingBlockInputStream(const BlockInputStreamPtr & input, SortDescription & description_sorted_, + SortDescription & description_to_sort_, + size_t max_merged_block_size_, size_t limit_); + + String getName() const override { return "FinishMergeSorting"; } + + bool isSortedOutput() const override { return true; } + const SortDescription & getSortDescription() const override { return description_to_sort; } + + Block getHeader() const override { return header; } + +protected: + Block readImpl() override; + +private: + SortDescription description_sorted; + SortDescription description_to_sort; + size_t max_merged_block_size; + size_t limit; + + Block tail_block; + Blocks blocks; + + std::unique_ptr impl; + + /// Before operation, will remove constant columns from blocks. And after, place constant columns back. + /// to avoid excessive virtual function calls + /// Save original block structure here. + Block header; + + bool end_of_stream = false; + size_t total_rows_processed = 0; +}; } diff --git a/dbms/src/Interpreters/sortBlock.cpp b/dbms/src/Interpreters/sortBlock.cpp index 3cf903f1cf1..9258857a0d4 100644 --- a/dbms/src/Interpreters/sortBlock.cpp +++ b/dbms/src/Interpreters/sortBlock.cpp @@ -13,9 +13,19 @@ namespace ErrorCodes } -using ColumnsWithSortDescriptions = std::vector>; +static inline bool needCollation(const IColumn * column, const SortColumnDescription & description) +{ + if (!description.collator) + return false; -static ColumnsWithSortDescriptions getColumnsWithSortDescription(const Block & block, const SortDescription & description) + if (!typeid_cast(column)) /// TODO Nullable(String) + throw Exception("Collations could be specified only for String columns.", ErrorCodes::BAD_COLLATION); + + return true; +} + + +ColumnsWithSortDescriptions getColumnsWithSortDescription(const Block & block, const SortDescription & description) { size_t size = description.size(); ColumnsWithSortDescriptions res; @@ -34,38 +44,6 @@ static ColumnsWithSortDescriptions getColumnsWithSortDescription(const Block & b } -static inline bool needCollation(const IColumn * column, const SortColumnDescription & description) -{ - if (!description.collator) - return false; - - if (!typeid_cast(column)) /// TODO Nullable(String) - throw Exception("Collations could be specified only for String columns.", ErrorCodes::BAD_COLLATION); - - return true; -} - - -struct PartialSortingLess -{ - const ColumnsWithSortDescriptions & columns; - - explicit PartialSortingLess(const ColumnsWithSortDescriptions & columns_) : columns(columns_) {} - - bool operator() (size_t a, size_t b) const - { - for (ColumnsWithSortDescriptions::const_iterator it = columns.begin(); it != columns.end(); ++it) - { - int res = it->second.direction * it->first->compareAt(a, b, *it->first, it->second.nulls_direction); - if (res < 0) - return true; - else if (res > 0) - return false; - } - return false; - } -}; - struct PartialSortingLessWithCollation { const ColumnsWithSortDescriptions & columns; diff --git a/dbms/src/Interpreters/sortBlock.h b/dbms/src/Interpreters/sortBlock.h index 7cd4824a904..cfb36ed5e4b 100644 --- a/dbms/src/Interpreters/sortBlock.h +++ b/dbms/src/Interpreters/sortBlock.h @@ -29,4 +29,28 @@ void stableGetPermutation(const Block & block, const SortDescription & descripti */ bool isAlreadySorted(const Block & block, const SortDescription & description); +using ColumnsWithSortDescriptions = std::vector>; + +struct PartialSortingLess +{ + const ColumnsWithSortDescriptions & columns; + + explicit PartialSortingLess(const ColumnsWithSortDescriptions & columns_) : columns(columns_) {} + + bool operator() (size_t a, size_t b) const + { + for (ColumnsWithSortDescriptions::const_iterator it = columns.begin(); it != columns.end(); ++it) + { + int res = it->second.direction * it->first->compareAt(a, b, *it->first, it->second.nulls_direction); + if (res < 0) + return true; + else if (res > 0) + return false; + } + return false; + } +}; + +ColumnsWithSortDescriptions getColumnsWithSortDescription(const Block & block, const SortDescription & description); + } From 8c2060b48a8c34e5e9988bd1295c89b3a0943d68 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Thu, 4 Oct 2018 17:55:02 +0300 Subject: [PATCH 29/79] wip on finish sorting --- .../MergeSortingBlockInputStream.cpp | 70 +++++++++++-------- .../MergeSortingBlockInputStream.h | 2 +- dbms/src/Interpreters/sortBlock.cpp | 21 ++++++ dbms/src/Interpreters/sortBlock.h | 20 ------ 4 files changed, 63 insertions(+), 50 deletions(-) diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp index 28c8c6c2094..198384cd7a3 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp @@ -319,17 +319,28 @@ FinishMergeSortingBlockInputStream::FinishMergeSortingBlockInputStream( removeConstantsFromSortDescription(header, description_to_sort); } -static bool equalKeysAt(const ColumnsWithSortDescriptions & lhs, const ColumnsWithSortDescriptions & rhs, size_t n, size_t m) -{ - for (auto it = lhs.begin(), jt = rhs.begin(); it != lhs.end(); ++it, ++jt) +struct Less +{ + const ColumnsWithSortDescriptions & left_columns; + const ColumnsWithSortDescriptions & right_columns; + + Less(const ColumnsWithSortDescriptions & left_columns_, const ColumnsWithSortDescriptions & right_columns_) : + left_columns(left_columns_), right_columns(right_columns_) {} + + bool operator() (size_t a, size_t b) const { - int res = it->first->compareAt(n, m, *jt->first, it->second.nulls_direction); - if (res != 0) - return false; + for (auto it = left_columns.begin(), jt = right_columns.begin(); it != left_columns.end(); ++it, ++jt) + { + int res = it->second.direction * it->first->compareAt(a, b, *jt->first, it->second.nulls_direction); + if (res < 0) + return true; + else if (res > 0) + return false; + } + return false; } - return true; -} +}; Block FinishMergeSortingBlockInputStream::readImpl() { @@ -356,7 +367,7 @@ Block FinishMergeSortingBlockInputStream::readImpl() { block = children.back()->read(); - /// End of input stream, but we can`t returns immediatly, we need to merge already read blocks. + /// End of input stream, but we can`t return immediatly, we need to merge already read blocks. /// Check it later, when get end of stream from impl. if (!block) { @@ -373,35 +384,36 @@ Block FinishMergeSortingBlockInputStream::readImpl() if (size == 0) continue; - auto columns_with_sort_desc = getColumnsWithSortDescription(block, description_sorted); - removeConstantsFromBlock(block); - /// May be new block starts with new key. + /// Find the position of last already read key in current block. if (!blocks.empty()) { const Block & last_block = blocks.back(); - if (!equalKeysAt(getColumnsWithSortDescription(last_block, description_sorted), columns_with_sort_desc, last_block.rows() - 1, 0)) + auto last_columns = getColumnsWithSortDescription(last_block, description_sorted); + auto current_columns = getColumnsWithSortDescription(block, description_sorted); + + Less less(last_columns, current_columns); + + IColumn::Permutation perm(size); + for (size_t i = 0; i < size; ++i) + perm[i] = i; + + auto it = std::upper_bound(perm.begin(), perm.end(), last_block.rows() - 1, less); + if (it != perm.end()) + { + tail_pos = it - perm.begin(); break; + } } - - IColumn::Permutation perm(size); - for (size_t i = 0; i < size; ++i) - perm[i] = i; - - PartialSortingLess less(columns_with_sort_desc); - - /// We need to save tail of block, because next block may starts with the same key as in tail - /// and we should sort these rows in one chunk. - tail_pos = *std::lower_bound(perm.begin(), perm.end(), size - 1, less); - - if (tail_pos != 0) - break; - /// If we reach here, that means that current block has all rows with the same key as tail of a previous block. + /// If we reach here, that means that current block is first in chunk + /// or it all consists of rows with the same key as tail of a previous block. blocks.push_back(block); } + /// We need to save tail of block, because next block may starts with the same key as in tail + /// and we should sort these rows in one chunk. if (block) { Block head_block = block.cloneEmpty(); @@ -411,8 +423,8 @@ Block FinishMergeSortingBlockInputStream::readImpl() head_block.getByPosition(i).column = block.getByPosition(i).column->cut(0, tail_pos); tail_block.getByPosition(i).column = block.getByPosition(i).column->cut(tail_pos, block.rows() - tail_pos); } - - blocks.push_back(head_block); + if (head_block.rows()) + blocks.push_back(head_block); } impl = std::make_unique(blocks, description_to_sort, max_merged_block_size, limit); diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.h b/dbms/src/DataStreams/MergeSortingBlockInputStream.h index 8a3bdda2750..076c19dc0a0 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.h +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.h @@ -133,7 +133,7 @@ private: /** Takes stream already sorted by `x` and finishes sorting it by (`x`, `y`). - * During sorting only blocks with rows equal by `x` saved in RAM. + * During sorting only blocks with rows that equal by `x` saved in RAM. * */ class FinishMergeSortingBlockInputStream : public IProfilingBlockInputStream { diff --git a/dbms/src/Interpreters/sortBlock.cpp b/dbms/src/Interpreters/sortBlock.cpp index 9258857a0d4..40c98dd7cd5 100644 --- a/dbms/src/Interpreters/sortBlock.cpp +++ b/dbms/src/Interpreters/sortBlock.cpp @@ -44,6 +44,27 @@ ColumnsWithSortDescriptions getColumnsWithSortDescription(const Block & block, c } +struct PartialSortingLess +{ + const ColumnsWithSortDescriptions & columns; + + explicit PartialSortingLess(const ColumnsWithSortDescriptions & columns_) : columns(columns_) {} + + bool operator() (size_t a, size_t b) const + { + for (ColumnsWithSortDescriptions::const_iterator it = columns.begin(); it != columns.end(); ++it) + { + int res = it->second.direction * it->first->compareAt(a, b, *it->first, it->second.nulls_direction); + if (res < 0) + return true; + else if (res > 0) + return false; + } + return false; + } +}; + + struct PartialSortingLessWithCollation { const ColumnsWithSortDescriptions & columns; diff --git a/dbms/src/Interpreters/sortBlock.h b/dbms/src/Interpreters/sortBlock.h index cfb36ed5e4b..d6bce9a715d 100644 --- a/dbms/src/Interpreters/sortBlock.h +++ b/dbms/src/Interpreters/sortBlock.h @@ -31,26 +31,6 @@ bool isAlreadySorted(const Block & block, const SortDescription & description); using ColumnsWithSortDescriptions = std::vector>; -struct PartialSortingLess -{ - const ColumnsWithSortDescriptions & columns; - - explicit PartialSortingLess(const ColumnsWithSortDescriptions & columns_) : columns(columns_) {} - - bool operator() (size_t a, size_t b) const - { - for (ColumnsWithSortDescriptions::const_iterator it = columns.begin(); it != columns.end(); ++it) - { - int res = it->second.direction * it->first->compareAt(a, b, *it->first, it->second.nulls_direction); - if (res < 0) - return true; - else if (res > 0) - return false; - } - return false; - } -}; - ColumnsWithSortDescriptions getColumnsWithSortDescription(const Block & block, const SortDescription & description); } From 5a01eb3d5e8570a5cf66488d7f0e544962c25d37 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 5 Oct 2018 17:35:35 +0300 Subject: [PATCH 30/79] add test --- dbms/src/DataStreams/tests/CMakeLists.txt | 3 + .../tests/finish_sorting_stream.cpp | 105 ++++++++++++++++++ 2 files changed, 108 insertions(+) create mode 100644 dbms/src/DataStreams/tests/finish_sorting_stream.cpp diff --git a/dbms/src/DataStreams/tests/CMakeLists.txt b/dbms/src/DataStreams/tests/CMakeLists.txt index 75657eed09e..3dbbc0edbe3 100644 --- a/dbms/src/DataStreams/tests/CMakeLists.txt +++ b/dbms/src/DataStreams/tests/CMakeLists.txt @@ -11,3 +11,6 @@ target_link_libraries (union_stream2 dbms) add_executable (collapsing_sorted_stream collapsing_sorted_stream.cpp ${SRCS}) target_link_libraries (collapsing_sorted_stream dbms) + +add_executable (finish_sorting_stream finish_sorting_stream.cpp ${SRCS}) +target_link_libraries (finish_sorting_stream dbms) diff --git a/dbms/src/DataStreams/tests/finish_sorting_stream.cpp b/dbms/src/DataStreams/tests/finish_sorting_stream.cpp new file mode 100644 index 00000000000..8e90f2cc1a5 --- /dev/null +++ b/dbms/src/DataStreams/tests/finish_sorting_stream.cpp @@ -0,0 +1,105 @@ +#include +#include + +#include +#include +#include + +#include +#include + +#include + +using namespace DB; + +int main(int argc, char ** argv) +{ + srand(123456); + + try + { + size_t m = argc >= 2 ? atoi(argv[1]) : 2; + size_t n = argc >= 3 ? atoi(argv[2]) : 10; + + Blocks blocks; + for (size_t t = 0; t < m; ++t) + { + Block block; + for (size_t i = 0; i < 2; ++i) + { + ColumnWithTypeAndName column; + column.name = "col" + std::to_string(i + 1); + column.type = std::make_shared(); + + auto col = ColumnInt32::create(); + auto & vec = col->getData(); + vec.resize(n); + + for (size_t j = 0; j < n; ++j) + vec[j] = rand() % 10; + + column.column = std::move(col); + block.insert(column); + } + blocks.push_back(block); + } + + SortDescription sort_descr; + sort_descr.emplace_back("col1", 1, 1); + + for (auto & block : blocks) + sortBlock(block, sort_descr); + + BlockInputStreamPtr stream = std::make_shared(blocks, sort_descr, n); + + SortDescription sort_descr_final; + sort_descr_final.emplace_back("col1", 1, 1); + sort_descr_final.emplace_back("col2", 1, 1); + + stream = std::make_shared(stream, sort_descr_final); + stream = std::make_shared(stream, sort_descr, sort_descr_final, n, 0); + + { + Stopwatch stopwatch; + stopwatch.start(); + + Block res = blocks[0].cloneEmpty(); + + while (Block block = stream->read()) + { + for (size_t i = 0; i < block.columns(); ++i) + { + MutableColumnPtr ptr = (*std::move(res.getByPosition(i).column)).mutate(); + ptr->insertRangeFrom(*block.getByPosition(i).column.get(), 0, block.rows()); + } + } + + if (res.rows() != n * m) + throw Exception("Result block size mismatch"); + + const auto & columns = res.getColumns(); + + for (size_t i = 1; i < res.rows(); ++i) + for (const auto & col : columns) + { + int res = col->compareAt(i - 1, i, *col, 1); + if (res < 0) + break; + else if (res > 0) + throw Exception("Result stream not sorted"); + } + + stopwatch.stop(); + std::cout << std::fixed << std::setprecision(2) + << "Elapsed " << stopwatch.elapsedSeconds() << " sec." + << ", " << n / stopwatch.elapsedSeconds() << " rows/sec." + << std::endl; + } + } + catch (const Exception & e) + { + std::cerr << e.displayText() << std::endl; + } + + return 0; +} From d4a56af66070b751e58abf24b3afb8e07d1e607d Mon Sep 17 00:00:00 2001 From: VadimPE Date: Wed, 10 Oct 2018 16:22:55 +0300 Subject: [PATCH 31/79] ISSUES-2581 fix bug with new partition --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 6 +++--- .../Storages/MergeTree/MergeTreeDataSelectExecutor.h | 4 ++-- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 12 ++++++------ 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 7ed38774a8e..d3c08248f55 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -137,7 +137,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read( const Context & context, const size_t max_block_size, const unsigned num_streams, - const PartitionIdToMaxBlock* max_blocks_number_to_read) const + const PartitionIdToMaxBlock * max_blocks_number_to_read) const { return readFromParts( data.getDataPartsVector(), column_names_to_return, query_info, context, @@ -151,7 +151,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( const Context & context, const size_t max_block_size, const unsigned num_streams, - const PartitionIdToMaxBlock* max_blocks_number_to_read) const + const PartitionIdToMaxBlock * max_blocks_number_to_read) const { size_t part_index = 0; @@ -274,7 +274,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( if (max_blocks_number_to_read) { auto blocks_iterator = max_blocks_number_to_read->find(part->info.partition_id); - if (blocks_iterator != max_blocks_number_to_read->end() && part->info.max_block > blocks_iterator->second) + if (blocks_iterator == max_blocks_number_to_read->end() || part->info.max_block > blocks_iterator->second) continue; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index d3715960fc2..75b6d7625d4 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -30,7 +30,7 @@ public: const Context & context, size_t max_block_size, unsigned num_streams, - const PartitionIdToMaxBlock* max_blocks_number_to_read = nullptr) const; + const PartitionIdToMaxBlock * max_blocks_number_to_read = nullptr) const; BlockInputStreams readFromParts( MergeTreeData::DataPartsVector parts, @@ -39,7 +39,7 @@ public: const Context & context, size_t max_block_size, unsigned num_streams, - const PartitionIdToMaxBlock* max_blocks_number_to_read = nullptr) const; + const PartitionIdToMaxBlock * max_blocks_number_to_read = nullptr) const; private: MergeTreeData & data; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index ac7d85aa9f1..67143a19c2a 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2952,12 +2952,13 @@ BlockInputStreams StorageReplicatedMergeTree::read( * 2. Do not read parts that have not yet been written to the quorum of the replicas. * For this you have to synchronously go to ZooKeeper. */ - ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock max_added_blocks; if (settings.select_sequential_consistency) { + ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock max_added_blocks; + for (const auto & data_part : data.getDataParts()) { - max_added_blocks[data_part->info.partition_id] = data_part->info.max_block; + max_added_blocks[data_part->info.partition_id] = std::max(max_added_blocks[data_part->info.partition_id], data_part->info.max_block); } auto zookeeper = getZooKeeper(); @@ -2993,12 +2994,11 @@ BlockInputStreams StorageReplicatedMergeTree::read( for (const auto & max_block : part_with_quorum.getMaxInsertedBlocks()) max_added_blocks[max_block.first] = max_block.second; } + + return reader.read(column_names, query_info, context, max_block_size, num_streams, &max_added_blocks); } - if (max_added_blocks.empty()) - return reader.read(column_names, query_info, context, max_block_size, num_streams); - else - return reader.read(column_names, query_info, context, max_block_size, num_streams, &max_added_blocks); + return reader.read(column_names, query_info, context, max_block_size, num_streams); } From cc2686cdf1f781a6e267b512a801f9c280f42a55 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Wed, 10 Oct 2018 19:13:14 +0300 Subject: [PATCH 32/79] ISSUES-2581 add tests --- .../00732_quorum_insert_zookeeper.reference | 7 ++++ .../00732_quorum_insert_zookeeper.sql | 34 +++++++++++++++++++ 2 files changed, 41 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.reference create mode 100644 dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.reference new file mode 100644 index 00000000000..3918ef82e26 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.reference @@ -0,0 +1,7 @@ +3 +3 +3 +3 +4 +4 + diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql new file mode 100644 index 00000000000..edb0d68864c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql @@ -0,0 +1,34 @@ +DROP TABLE IF EXISTS test.quorum1; +DROP TABLE IF EXISTS test.quorum2; + +CREATE TABLE test.quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/fetches', '1') ORDER BY x PARTITION BY y; +CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/fetches', '2') ORDER BY x PARTITION BY y; + +SET insert_quorum=2; +SET select_sequential_consistency=1; +SET insert_quorum_timeout=0; + +INSERT INTO test.quorum1 VALUES (1, 1); +INSERT INTO test.quorum1 VALUES (2, 1); +INSERT INTO test.quorum1 VALUES (1, 100); + +SELECT count(*) FROM test.quorum1; +SELECT count(*) FROM test.quorum2; + +SYSTEM STOP FETCHES test.quorum1; + + +INSERT INTO test.quorum2 VALUES (1, 200); -- { serverError 319 } -- + +SELECT count(*) FROM test.quorum1; +SELECT count(*) FROM test.quorum2; + + +SYSTEM START FETCHES test.quorum1; + +SYSTEM SYNC REPLICA test.quorum1; + + +SELECT count(*) FROM test.quorum1; +SELECT count(*) FROM test.quorum2; + From 4db6eab5a3f765f5fee5c6caebdf3e831ed8f1e8 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Wed, 10 Oct 2018 19:20:15 +0300 Subject: [PATCH 33/79] ISSUES-2581 fix name --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 12 ++++++------ .../Storages/MergeTree/MergeTreeDataSelectExecutor.h | 6 +++--- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index d3c08248f55..70a32c3a8c3 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -137,11 +137,11 @@ BlockInputStreams MergeTreeDataSelectExecutor::read( const Context & context, const size_t max_block_size, const unsigned num_streams, - const PartitionIdToMaxBlock * max_blocks_number_to_read) const + const PartitionIdToMaxBlock * max_block_numbers_to_read) const { return readFromParts( data.getDataPartsVector(), column_names_to_return, query_info, context, - max_block_size, num_streams, max_blocks_number_to_read); + max_block_size, num_streams, max_block_numbers_to_read); } BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( @@ -151,7 +151,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( const Context & context, const size_t max_block_size, const unsigned num_streams, - const PartitionIdToMaxBlock * max_blocks_number_to_read) const + const PartitionIdToMaxBlock * max_block_numbers_to_read) const { size_t part_index = 0; @@ -271,10 +271,10 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( part->minmax_idx.parallelogram, data.minmax_idx_column_types)) continue; - if (max_blocks_number_to_read) + if (max_block_numbers_to_read) { - auto blocks_iterator = max_blocks_number_to_read->find(part->info.partition_id); - if (blocks_iterator == max_blocks_number_to_read->end() || part->info.max_block > blocks_iterator->second) + auto blocks_iterator = max_block_numbers_to_read->find(part->info.partition_id); + if (blocks_iterator == max_block_numbers_to_read->end() || part->info.max_block > blocks_iterator->second) continue; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 75b6d7625d4..8ea0a44d1b7 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -20,7 +20,7 @@ public: MergeTreeDataSelectExecutor(MergeTreeData & data_); /** When reading, selects a set of parts that covers the desired range of the index. - * max_blocks_number_to_read - if not empty, do not read all the parts whose right border is greater than max_block in partition. + * max_blocks_number_to_read - if not nullptr, do not read all the parts whose right border is greater than max_block in partition. */ using PartitionIdToMaxBlock = std::unordered_map; @@ -30,7 +30,7 @@ public: const Context & context, size_t max_block_size, unsigned num_streams, - const PartitionIdToMaxBlock * max_blocks_number_to_read = nullptr) const; + const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; BlockInputStreams readFromParts( MergeTreeData::DataPartsVector parts, @@ -39,7 +39,7 @@ public: const Context & context, size_t max_block_size, unsigned num_streams, - const PartitionIdToMaxBlock * max_blocks_number_to_read = nullptr) const; + const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; private: MergeTreeData & data; From 8c5be3aa679bf18c72e460d10776c12fdf7601e9 Mon Sep 17 00:00:00 2001 From: Vadim Date: Wed, 10 Oct 2018 19:21:46 +0300 Subject: [PATCH 34/79] Update 00732_quorum_insert_zookeeper.reference --- .../queries/0_stateless/00732_quorum_insert_zookeeper.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.reference index 3918ef82e26..dda5c8f7cad 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.reference @@ -4,4 +4,3 @@ 3 4 4 - From 93992879bcebf4e58819705aeec01e937e3c4887 Mon Sep 17 00:00:00 2001 From: Vadim Date: Wed, 10 Oct 2018 19:22:06 +0300 Subject: [PATCH 35/79] Update 00732_quorum_insert_zookeeper.sql --- dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql index edb0d68864c..c6cff82bd23 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql @@ -31,4 +31,3 @@ SYSTEM SYNC REPLICA test.quorum1; SELECT count(*) FROM test.quorum1; SELECT count(*) FROM test.quorum2; - From 25b100c1f0da3cf7ee4e988d3cf1e9138019ae45 Mon Sep 17 00:00:00 2001 From: Vadim Date: Wed, 10 Oct 2018 19:23:50 +0300 Subject: [PATCH 36/79] Update StorageReplicatedMergeTree.cpp --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 67143a19c2a..bcf02abc412 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2986,7 +2986,7 @@ BlockInputStreams StorageReplicatedMergeTree::read( ReplicatedMergeTreeQuorumAddedParts part_with_quorum(added_parts_str, data.format_version); auto added_parts = part_with_quorum.added_parts; - for (auto & added_part : added_parts) + for (const auto & added_part : added_parts) if (!data.getActiveContainingPart(added_part.second)) throw Exception("Replica doesn't have part " + added_part.second + " which was successfully written to quorum of other replicas." " Send query to another replica or disable 'select_sequential_consistency' setting.", ErrorCodes::REPLICA_IS_NOT_IN_QUORUM); From 6b2bfd2f4f7f27b1e1c42a3430c58a64fce57b90 Mon Sep 17 00:00:00 2001 From: Vadim Date: Wed, 10 Oct 2018 20:35:34 +0300 Subject: [PATCH 37/79] Update 00732_quorum_insert_zookeeper.sql --- .../queries/0_stateless/00732_quorum_insert_zookeeper.sql | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql index c6cff82bd23..c66d3ad81b5 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS test.quorum1; DROP TABLE IF EXISTS test.quorum2; -CREATE TABLE test.quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/fetches', '1') ORDER BY x PARTITION BY y; -CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/fetches', '2') ORDER BY x PARTITION BY y; +CREATE TABLE test.quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '1') ORDER BY x PARTITION BY y; +CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '2') ORDER BY x PARTITION BY y; SET insert_quorum=2; SET select_sequential_consistency=1; @@ -31,3 +31,6 @@ SYSTEM SYNC REPLICA test.quorum1; SELECT count(*) FROM test.quorum1; SELECT count(*) FROM test.quorum2; + +DROP TABLE IF EXISTS test.quorum1; +DROP TABLE IF EXISTS test.quorum2; From 517619d8c13d0bab5a1f56ad6b8905a449e101d4 Mon Sep 17 00:00:00 2001 From: Vadim Date: Wed, 10 Oct 2018 20:49:42 +0300 Subject: [PATCH 38/79] Update 00732_quorum_insert_zookeeper.sql --- .../queries/0_stateless/00732_quorum_insert_zookeeper.sql | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql index c66d3ad81b5..cb71b5b872a 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql @@ -6,7 +6,6 @@ CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickho SET insert_quorum=2; SET select_sequential_consistency=1; -SET insert_quorum_timeout=0; INSERT INTO test.quorum1 VALUES (1, 1); INSERT INTO test.quorum1 VALUES (2, 1); @@ -15,20 +14,20 @@ INSERT INTO test.quorum1 VALUES (1, 100); SELECT count(*) FROM test.quorum1; SELECT count(*) FROM test.quorum2; -SYSTEM STOP FETCHES test.quorum1; +SET insert_quorum_timeout=0; +SYSTEM STOP FETCHES test.quorum1; INSERT INTO test.quorum2 VALUES (1, 200); -- { serverError 319 } -- SELECT count(*) FROM test.quorum1; SELECT count(*) FROM test.quorum2; +SET insert_quorum_timeout=100; SYSTEM START FETCHES test.quorum1; - SYSTEM SYNC REPLICA test.quorum1; - SELECT count(*) FROM test.quorum1; SELECT count(*) FROM test.quorum2; From c565c75350de22ac21cf7cfac12b6fce7442b800 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Thu, 11 Oct 2018 12:27:16 +0300 Subject: [PATCH 39/79] fix style --- .../MergeSortingBlockInputStream.cpp | 24 +++++++++---------- .../MergeSortingBlockInputStream.h | 8 +++---- .../tests/finish_sorting_stream.cpp | 18 +++++++------- 3 files changed, 25 insertions(+), 25 deletions(-) diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp index 198384cd7a3..999b554fb0f 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp @@ -306,9 +306,9 @@ void MergeSortingBlockInputStream::remerge() } -FinishMergeSortingBlockInputStream::FinishMergeSortingBlockInputStream( +FinishSortingBlockInputStream::FinishSortingBlockInputStream( const BlockInputStreamPtr & input, SortDescription & description_sorted_, - SortDescription & description_to_sort_, + SortDescription & description_to_sort_, size_t max_merged_block_size_, size_t limit_) : description_sorted(description_sorted_), description_to_sort(description_to_sort_), max_merged_block_size(max_merged_block_size_), limit(limit_) @@ -325,7 +325,7 @@ struct Less const ColumnsWithSortDescriptions & left_columns; const ColumnsWithSortDescriptions & right_columns; - Less(const ColumnsWithSortDescriptions & left_columns_, const ColumnsWithSortDescriptions & right_columns_) : + Less(const ColumnsWithSortDescriptions & left_columns_, const ColumnsWithSortDescriptions & right_columns_) : left_columns(left_columns_), right_columns(right_columns_) {} bool operator() (size_t a, size_t b) const @@ -342,7 +342,7 @@ struct Less } }; -Block FinishMergeSortingBlockInputStream::readImpl() +Block FinishSortingBlockInputStream::readImpl() { if (limit && total_rows_processed == limit) return {}; @@ -393,12 +393,12 @@ Block FinishMergeSortingBlockInputStream::readImpl() auto last_columns = getColumnsWithSortDescription(last_block, description_sorted); auto current_columns = getColumnsWithSortDescription(block, description_sorted); - Less less(last_columns, current_columns); - + Less less(last_columns, current_columns); + IColumn::Permutation perm(size); for (size_t i = 0; i < size; ++i) perm[i] = i; - + auto it = std::upper_bound(perm.begin(), perm.end(), last_block.rows() - 1, less); if (it != perm.end()) { @@ -406,8 +406,8 @@ Block FinishMergeSortingBlockInputStream::readImpl() break; } } - - /// If we reach here, that means that current block is first in chunk + + /// If we reach here, that means that current block is first in chunk /// or it all consists of rows with the same key as tail of a previous block. blocks.push_back(block); } @@ -424,7 +424,7 @@ Block FinishMergeSortingBlockInputStream::readImpl() tail_block.getByPosition(i).column = block.getByPosition(i).column->cut(tail_pos, block.rows() - tail_pos); } if (head_block.rows()) - blocks.push_back(head_block); + blocks.push_back(head_block); } impl = std::make_unique(blocks, description_to_sort, max_merged_block_size, limit); @@ -433,9 +433,9 @@ Block FinishMergeSortingBlockInputStream::readImpl() if (res) enrichBlockWithConstants(res, header); - + total_rows_processed += res.rows(); - + return res; } diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.h b/dbms/src/DataStreams/MergeSortingBlockInputStream.h index 076c19dc0a0..9b1a590e5f7 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.h +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.h @@ -132,15 +132,15 @@ private: }; -/** Takes stream already sorted by `x` and finishes sorting it by (`x`, `y`). +/** Takes stream already sorted by `x` and finishes sorting it by (`x`, `y`). * During sorting only blocks with rows that equal by `x` saved in RAM. * */ -class FinishMergeSortingBlockInputStream : public IProfilingBlockInputStream +class FinishSortingBlockInputStream : public IProfilingBlockInputStream { public: /// limit - if not 0, allowed to return just first 'limit' rows in sorted order. - FinishMergeSortingBlockInputStream(const BlockInputStreamPtr & input, SortDescription & description_sorted_, - SortDescription & description_to_sort_, + FinishSortingBlockInputStream(const BlockInputStreamPtr & input, SortDescription & description_sorted_, + SortDescription & description_to_sort_, size_t max_merged_block_size_, size_t limit_); String getName() const override { return "FinishMergeSorting"; } diff --git a/dbms/src/DataStreams/tests/finish_sorting_stream.cpp b/dbms/src/DataStreams/tests/finish_sorting_stream.cpp index 8e90f2cc1a5..39be70715df 100644 --- a/dbms/src/DataStreams/tests/finish_sorting_stream.cpp +++ b/dbms/src/DataStreams/tests/finish_sorting_stream.cpp @@ -49,7 +49,7 @@ int main(int argc, char ** argv) for (auto & block : blocks) sortBlock(block, sort_descr); - + BlockInputStreamPtr stream = std::make_shared(blocks, sort_descr, n); SortDescription sort_descr_final; @@ -57,29 +57,29 @@ int main(int argc, char ** argv) sort_descr_final.emplace_back("col2", 1, 1); stream = std::make_shared(stream, sort_descr_final); - stream = std::make_shared(stream, sort_descr, sort_descr_final, n, 0); - + stream = std::make_shared(stream, sort_descr, sort_descr_final, n, 0); + { Stopwatch stopwatch; stopwatch.start(); - Block res = blocks[0].cloneEmpty(); + Block res_block = blocks[0].cloneEmpty(); while (Block block = stream->read()) { for (size_t i = 0; i < block.columns(); ++i) { - MutableColumnPtr ptr = (*std::move(res.getByPosition(i).column)).mutate(); + MutableColumnPtr ptr = (*std::move(res_block.getByPosition(i).column)).mutate(); ptr->insertRangeFrom(*block.getByPosition(i).column.get(), 0, block.rows()); } } - if (res.rows() != n * m) + if (res_block.rows() != n * m) throw Exception("Result block size mismatch"); - const auto & columns = res.getColumns(); + const auto & columns = res_block.getColumns(); - for (size_t i = 1; i < res.rows(); ++i) + for (size_t i = 1; i < res_block.rows(); ++i) for (const auto & col : columns) { int res = col->compareAt(i - 1, i, *col, 1); @@ -88,7 +88,7 @@ int main(int argc, char ** argv) else if (res > 0) throw Exception("Result stream not sorted"); } - + stopwatch.stop(); std::cout << std::fixed << std::setprecision(2) << "Elapsed " << stopwatch.elapsedSeconds() << " sec." From 1c51407fa721b2ed0e386ffed0f8515804e82417 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Mon, 15 Oct 2018 13:27:17 +0300 Subject: [PATCH 40/79] ISSUES-2581 add test --- .../00732_quorum_insert_zookeeper.reference | 6 ---- .../00732_quorum_insert_zookeeper.sql | 35 ------------------- 2 files changed, 41 deletions(-) delete mode 100644 dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.reference delete mode 100644 dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.reference deleted file mode 100644 index dda5c8f7cad..00000000000 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.reference +++ /dev/null @@ -1,6 +0,0 @@ -3 -3 -3 -3 -4 -4 diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql deleted file mode 100644 index cb71b5b872a..00000000000 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_zookeeper.sql +++ /dev/null @@ -1,35 +0,0 @@ -DROP TABLE IF EXISTS test.quorum1; -DROP TABLE IF EXISTS test.quorum2; - -CREATE TABLE test.quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '1') ORDER BY x PARTITION BY y; -CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '2') ORDER BY x PARTITION BY y; - -SET insert_quorum=2; -SET select_sequential_consistency=1; - -INSERT INTO test.quorum1 VALUES (1, 1); -INSERT INTO test.quorum1 VALUES (2, 1); -INSERT INTO test.quorum1 VALUES (1, 100); - -SELECT count(*) FROM test.quorum1; -SELECT count(*) FROM test.quorum2; - -SET insert_quorum_timeout=0; - -SYSTEM STOP FETCHES test.quorum1; - -INSERT INTO test.quorum2 VALUES (1, 200); -- { serverError 319 } -- - -SELECT count(*) FROM test.quorum1; -SELECT count(*) FROM test.quorum2; - -SET insert_quorum_timeout=100; - -SYSTEM START FETCHES test.quorum1; -SYSTEM SYNC REPLICA test.quorum1; - -SELECT count(*) FROM test.quorum1; -SELECT count(*) FROM test.quorum2; - -DROP TABLE IF EXISTS test.quorum1; -DROP TABLE IF EXISTS test.quorum2; From 58c2bd366f0e55f7c7962e72269466031aae00c8 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Mon, 15 Oct 2018 13:31:24 +0300 Subject: [PATCH 41/79] ISSUES-2581 add test --- ...st_part_and_alive_part_zookeeper.reference | 6 +++ ...ert_lost_part_and_alive_part_zookeeper.sql | 40 +++++++++++++++++++ ...uorum_insert_lost_part_zookeeper.reference | 5 +++ ...0732_quorum_insert_lost_part_zookeeper.sql | 36 +++++++++++++++++ ...rt_simple_test_1_parts_zookeeper.reference | 3 ++ ...m_insert_simple_test_1_parts_zookeeper.sql | 19 +++++++++ ...rt_simple_test_2_parts_zookeeper.reference | 4 ++ ...m_insert_simple_test_2_parts_zookeeper.sql | 20 ++++++++++ 8 files changed, 133 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference create mode 100644 dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql create mode 100644 dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.reference create mode 100644 dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql create mode 100644 dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference create mode 100644 dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql create mode 100644 dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.reference create mode 100644 dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference new file mode 100644 index 00000000000..ee1dc968732 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference @@ -0,0 +1,6 @@ +3 +3 +4 +4 +4 + diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql new file mode 100644 index 00000000000..8a64b844043 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql @@ -0,0 +1,40 @@ +SET send_logs_level = 'none'; + +DROP TABLE IF EXISTS test.quorum1; +DROP TABLE IF EXISTS test.quorum2; + +CREATE TABLE test.quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '1') ORDER BY x PARTITION BY y; +CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '2') ORDER BY x PARTITION BY y; + +SET insert_quorum=2; +SET select_sequential_consistency=1; + +INSERT INTO test.quorum2 VALUES (1, '15-10-2018'); +INSERT INTO test.quorum2 VALUES (2, '15-10-2018'); +INSERT INTO test.quorum2 VALUES (2, '15-11-2018'); + +SET insert_quorum_timeout=100; + +SYSTEM STOP FETCHES test.quorum1; + +INSERT INTO test.quorum2 VALUES (1, '15-12-2018'); -- { serverError 319 } -- + +SELECT count(*) FROM test.quorum1; +SELECT count(*) FROM test.quorum2; + +SET select_sequential_consistency=0; + +SELECT count(*) FROM test.quorum2; + +SET select_sequential_consistency=1; + +SET insert_quorum_timeout=100; + +SYSTEM START FETCHES test.quorum1; +SYSTEM SYNC REPLICA test.quorum1; + +SELECT count(*) FROM test.quorum1; +SELECT count(*) FROM test.quorum2; + +DROP TABLE IF EXISTS test.quorum1; +DROP TABLE IF EXISTS test.quorum2; diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.reference new file mode 100644 index 00000000000..f6ecf659a54 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.reference @@ -0,0 +1,5 @@ +0 +0 +1 +1 +1 diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql new file mode 100644 index 00000000000..7a7ffa1de87 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql @@ -0,0 +1,36 @@ +SET send_logs_level = 'none'; + +DROP TABLE IF EXISTS test.quorum1; +DROP TABLE IF EXISTS test.quorum2; + +CREATE TABLE test.quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '1') ORDER BY x PARTITION BY y; +CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '2') ORDER BY x PARTITION BY y; + +SET insert_quorum=2; +SET select_sequential_consistency=1; + +SET insert_quorum_timeout=0; + +SYSTEM STOP FETCHES test.quorum1; + +INSERT INTO test.quorum2 VALUES (1, '15-11-2018'); -- { serverError 319 } -- + +SELECT count(*) FROM test.quorum1; +SELECT count(*) FROM test.quorum2; + +SET select_sequential_consistency=0; + +SELECT count(*) FROM test.quorum2; + +SET select_sequential_consistency=1; + +SET insert_quorum_timeout=100; + +SYSTEM START FETCHES test.quorum1; +SYSTEM SYNC REPLICA test.quorum1; + +SELECT count(*) FROM test.quorum1; +SELECT count(*) FROM test.quorum2; + +DROP TABLE IF EXISTS test.quorum1; +DROP TABLE IF EXISTS test.quorum2; diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference new file mode 100644 index 00000000000..2cfac62750e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference @@ -0,0 +1,3 @@ +2 +2 + diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql new file mode 100644 index 00000000000..48e0cd62be2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql @@ -0,0 +1,19 @@ +SET send_logs_level = 'none'; + +DROP TABLE IF EXISTS test.quorum1; +DROP TABLE IF EXISTS test.quorum2; + +CREATE TABLE test.quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '1') ORDER BY x PARTITION BY y; +CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '2') ORDER BY x PARTITION BY y; + +SET insert_quorum=2; +SET select_sequential_consistency=1; + +INSERT INTO test.quorum1 VALUES (1, '15-11-2018'); +INSERT INTO test.quorum1 VALUES (2, '15-11-2018'); + +SELECT count(*) FROM test.quorum1; +SELECT count(*) FROM test.quorum2; + +DROP TABLE IF EXISTS test.quorum1; +DROP TABLE IF EXISTS test.quorum2; diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.reference new file mode 100644 index 00000000000..98c07c27497 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.reference @@ -0,0 +1,4 @@ +3 +3 + + diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql new file mode 100644 index 00000000000..f37d1d82669 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql @@ -0,0 +1,20 @@ +SET send_logs_level = 'none'; + +DROP TABLE IF EXISTS test.quorum1; +DROP TABLE IF EXISTS test.quorum2; + +CREATE TABLE test.quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/fetches', '1') ORDER BY x PARTITION BY y; +CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/fetches', '2') ORDER BY x PARTITION BY y; + +SET insert_quorum=2; +SET select_sequential_consistency=1; + +INSERT INTO test.quorum1 VALUES (1, '15-11-2018'); +INSERT INTO test.quorum1 VALUES (2, '15-11-2018'); +INSERT INTO test.quorum1 VALUES (1, '15-12-2018'); + +SELECT count(*) FROM test.quorum1; +SELECT count(*) FROM test.quorum2; + +DROP TABLE IF EXISTS test.quorum1; +DROP TABLE IF EXISTS test.quorum2; From 9b1271afd37df75bdbeac83e3461ffd784d6781c Mon Sep 17 00:00:00 2001 From: Vadim Date: Mon, 15 Oct 2018 13:32:17 +0300 Subject: [PATCH 42/79] Update 00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference --- ...32_quorum_insert_lost_part_and_alive_part_zookeeper.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference index ee1dc968732..336921101fa 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference @@ -3,4 +3,3 @@ 4 4 4 - From 5898a2282488f7938d51cbfe7b717709d45a61bf Mon Sep 17 00:00:00 2001 From: Vadim Date: Mon, 15 Oct 2018 13:32:30 +0300 Subject: [PATCH 43/79] Update 00732_quorum_insert_simple_test_1_parts_zookeeper.reference --- .../00732_quorum_insert_simple_test_1_parts_zookeeper.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference index 2cfac62750e..51993f072d5 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference @@ -1,3 +1,2 @@ 2 2 - From a013dce567f2e406d29faa208f4e533e6af5df21 Mon Sep 17 00:00:00 2001 From: Vadim Date: Mon, 15 Oct 2018 13:32:43 +0300 Subject: [PATCH 44/79] Update 00732_quorum_insert_simple_test_2_parts_zookeeper.reference --- .../00732_quorum_insert_simple_test_2_parts_zookeeper.reference | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.reference index 98c07c27497..a5c8806279f 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.reference @@ -1,4 +1,2 @@ 3 3 - - From 7b6f0f5372323d7746c943b29d4003e3b6f839f0 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Mon, 15 Oct 2018 15:53:05 +0300 Subject: [PATCH 45/79] CLICKHOUSE-1791 add modulo for date --- dbms/src/Functions/FunctionsArithmetic.h | 6 +++++- .../queries/0_stateless/00726_modulo_for_date.reference | 6 ++++++ dbms/tests/queries/0_stateless/00726_modulo_for_date.sql | 6 ++++++ 3 files changed, 17 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/00726_modulo_for_date.reference create mode 100644 dbms/tests/queries/0_stateless/00726_modulo_for_date.sql diff --git a/dbms/src/Functions/FunctionsArithmetic.h b/dbms/src/Functions/FunctionsArithmetic.h index 1cfc7f9ab56..31817d988f0 100644 --- a/dbms/src/Functions/FunctionsArithmetic.h +++ b/dbms/src/Functions/FunctionsArithmetic.h @@ -1055,7 +1055,11 @@ public: /// least(Date, Date) -> Date /// greatest(Date, Date) -> Date Case && (std::is_same_v> || std::is_same_v>), - LeftDataType>>; + LeftDataType>, + /// Date % Int32 -> int32 + Case>, Switch< + Case && IsIntegral, RightDataType>>>>; + }; diff --git a/dbms/tests/queries/0_stateless/00726_modulo_for_date.reference b/dbms/tests/queries/0_stateless/00726_modulo_for_date.reference new file mode 100644 index 00000000000..ae6acaeb9f7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00726_modulo_for_date.reference @@ -0,0 +1,6 @@ +SELECT toDate('21-06-2018') % 234 = toInt16(toDate('21-06-2018')) % 234; +SELECT toDate('21-06-2018') % 23456 = toInt16(toDate('21-06-2018')) % 23456; +SELECT toDate('21-06-2018') % 12376 = toInt16(toDate('21-06-2018')) % 12376; +SELECT toDate('21-06-2018 12:12:12') % 234 = toInt32(toDate('21-06-2018 12:12:12')) % 234; +SELECT toDate('21-06-2018 12:12:12') % 23456 = toInt32(toDate('21-06-2018 12:12:12')) % 23456; +SELECT toDate('21-06-2018 12:12:12') % 12376 = toInt32(toDate('21-06-2018 12:12:12')) % 12376; diff --git a/dbms/tests/queries/0_stateless/00726_modulo_for_date.sql b/dbms/tests/queries/0_stateless/00726_modulo_for_date.sql new file mode 100644 index 00000000000..ae6acaeb9f7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00726_modulo_for_date.sql @@ -0,0 +1,6 @@ +SELECT toDate('21-06-2018') % 234 = toInt16(toDate('21-06-2018')) % 234; +SELECT toDate('21-06-2018') % 23456 = toInt16(toDate('21-06-2018')) % 23456; +SELECT toDate('21-06-2018') % 12376 = toInt16(toDate('21-06-2018')) % 12376; +SELECT toDate('21-06-2018 12:12:12') % 234 = toInt32(toDate('21-06-2018 12:12:12')) % 234; +SELECT toDate('21-06-2018 12:12:12') % 23456 = toInt32(toDate('21-06-2018 12:12:12')) % 23456; +SELECT toDate('21-06-2018 12:12:12') % 12376 = toInt32(toDate('21-06-2018 12:12:12')) % 12376; From e331f15ffd96eb03be6087a728463ac9164f7bcb Mon Sep 17 00:00:00 2001 From: VadimPE Date: Mon, 15 Oct 2018 16:24:01 +0300 Subject: [PATCH 46/79] CLICKHOUSE-1791 add modulo for nonIntegral divisor --- dbms/src/Functions/FunctionsArithmetic.h | 7 ++++++- .../00726_modulo_for_date.reference | 18 ++++++++++++------ .../0_stateless/00726_modulo_for_date.sql | 6 ++++++ 3 files changed, 24 insertions(+), 7 deletions(-) diff --git a/dbms/src/Functions/FunctionsArithmetic.h b/dbms/src/Functions/FunctionsArithmetic.h index 31817d988f0..19495fc3ec4 100644 --- a/dbms/src/Functions/FunctionsArithmetic.h +++ b/dbms/src/Functions/FunctionsArithmetic.h @@ -997,6 +997,10 @@ template <> constexpr bool IsIntegral = true; template <> constexpr bool IsIntegral = true; template <> constexpr bool IsIntegral = true; +template constexpr bool IsNonIntegral = false; +template <> constexpr bool IsNonIntegral = true; +template <> constexpr bool IsNonIntegral = true; + template constexpr bool IsDateOrDateTime = false; template <> constexpr bool IsDateOrDateTime = true; template <> constexpr bool IsDateOrDateTime = true; @@ -1058,7 +1062,8 @@ public: LeftDataType>, /// Date % Int32 -> int32 Case>, Switch< - Case && IsIntegral, RightDataType>>>>; + Case && IsIntegral, RightDataType>, + Case && IsNonIntegral, DataTypeInt32>>>>; }; diff --git a/dbms/tests/queries/0_stateless/00726_modulo_for_date.reference b/dbms/tests/queries/0_stateless/00726_modulo_for_date.reference index ae6acaeb9f7..4dff9ef38ef 100644 --- a/dbms/tests/queries/0_stateless/00726_modulo_for_date.reference +++ b/dbms/tests/queries/0_stateless/00726_modulo_for_date.reference @@ -1,6 +1,12 @@ -SELECT toDate('21-06-2018') % 234 = toInt16(toDate('21-06-2018')) % 234; -SELECT toDate('21-06-2018') % 23456 = toInt16(toDate('21-06-2018')) % 23456; -SELECT toDate('21-06-2018') % 12376 = toInt16(toDate('21-06-2018')) % 12376; -SELECT toDate('21-06-2018 12:12:12') % 234 = toInt32(toDate('21-06-2018 12:12:12')) % 234; -SELECT toDate('21-06-2018 12:12:12') % 23456 = toInt32(toDate('21-06-2018 12:12:12')) % 23456; -SELECT toDate('21-06-2018 12:12:12') % 12376 = toInt32(toDate('21-06-2018 12:12:12')) % 12376; +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/dbms/tests/queries/0_stateless/00726_modulo_for_date.sql b/dbms/tests/queries/0_stateless/00726_modulo_for_date.sql index ae6acaeb9f7..45dbdd3e953 100644 --- a/dbms/tests/queries/0_stateless/00726_modulo_for_date.sql +++ b/dbms/tests/queries/0_stateless/00726_modulo_for_date.sql @@ -4,3 +4,9 @@ SELECT toDate('21-06-2018') % 12376 = toInt16(toDate('21-06-2018')) % 12376; SELECT toDate('21-06-2018 12:12:12') % 234 = toInt32(toDate('21-06-2018 12:12:12')) % 234; SELECT toDate('21-06-2018 12:12:12') % 23456 = toInt32(toDate('21-06-2018 12:12:12')) % 23456; SELECT toDate('21-06-2018 12:12:12') % 12376 = toInt32(toDate('21-06-2018 12:12:12')) % 12376; +SELECT toDate('21-06-2018') % 234 = toInt16(toDate('21-06-2018')) % 234.8; +SELECT toDate('21-06-2018') % 23456 = toInt16(toDate('21-06-2018')) % 23456.8; +SELECT toDate('21-06-2018') % 12376 = toInt16(toDate('21-06-2018')) % 12376.8; +SELECT toDate('21-06-2018 12:12:12') % 234 = toInt32(toDate('21-06-2018 12:12:12')) % 234.8; +SELECT toDate('21-06-2018 12:12:12') % 23456 = toInt32(toDate('21-06-2018 12:12:12')) % 23456.8; +SELECT toDate('21-06-2018 12:12:12') % 12376 = toInt32(toDate('21-06-2018 12:12:12')) % 12376.8; From ade4f56c3f9628fdea71f899544423c170a61857 Mon Sep 17 00:00:00 2001 From: Vadim Date: Mon, 15 Oct 2018 16:24:46 +0300 Subject: [PATCH 47/79] Update 00726_modulo_for_date.sql --- .../queries/0_stateless/00726_modulo_for_date.sql | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00726_modulo_for_date.sql b/dbms/tests/queries/0_stateless/00726_modulo_for_date.sql index 45dbdd3e953..d7ae1e4545a 100644 --- a/dbms/tests/queries/0_stateless/00726_modulo_for_date.sql +++ b/dbms/tests/queries/0_stateless/00726_modulo_for_date.sql @@ -4,9 +4,9 @@ SELECT toDate('21-06-2018') % 12376 = toInt16(toDate('21-06-2018')) % 12376; SELECT toDate('21-06-2018 12:12:12') % 234 = toInt32(toDate('21-06-2018 12:12:12')) % 234; SELECT toDate('21-06-2018 12:12:12') % 23456 = toInt32(toDate('21-06-2018 12:12:12')) % 23456; SELECT toDate('21-06-2018 12:12:12') % 12376 = toInt32(toDate('21-06-2018 12:12:12')) % 12376; -SELECT toDate('21-06-2018') % 234 = toInt16(toDate('21-06-2018')) % 234.8; -SELECT toDate('21-06-2018') % 23456 = toInt16(toDate('21-06-2018')) % 23456.8; -SELECT toDate('21-06-2018') % 12376 = toInt16(toDate('21-06-2018')) % 12376.8; -SELECT toDate('21-06-2018 12:12:12') % 234 = toInt32(toDate('21-06-2018 12:12:12')) % 234.8; -SELECT toDate('21-06-2018 12:12:12') % 23456 = toInt32(toDate('21-06-2018 12:12:12')) % 23456.8; -SELECT toDate('21-06-2018 12:12:12') % 12376 = toInt32(toDate('21-06-2018 12:12:12')) % 12376.8; +SELECT toDate('21-06-2018') % 234.8 = toInt16(toDate('21-06-2018')) % 234.8; +SELECT toDate('21-06-2018') % 23456.8 = toInt16(toDate('21-06-2018')) % 23456.8; +SELECT toDate('21-06-2018') % 12376.8 = toInt16(toDate('21-06-2018')) % 12376.8; +SELECT toDate('21-06-2018 12:12:12') % 234.8 = toInt32(toDate('21-06-2018 12:12:12')) % 234.8; +SELECT toDate('21-06-2018 12:12:12') % 23456.8 = toInt32(toDate('21-06-2018 12:12:12')) % 23456.8; +SELECT toDate('21-06-2018 12:12:12') % 12376.8 = toInt32(toDate('21-06-2018 12:12:12')) % 12376.8; From 4d6da001844ee8378ca3bcfdcaf5a1f804789a47 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Mon, 15 Oct 2018 17:09:22 +0300 Subject: [PATCH 48/79] fix tests --- ...rum_insert_lost_part_and_alive_part_zookeeper.sql | 12 ++++++------ .../00732_quorum_insert_lost_part_zookeeper.sql | 2 +- ...2_quorum_insert_simple_test_1_parts_zookeeper.sql | 4 ++-- ...2_quorum_insert_simple_test_2_parts_zookeeper.sql | 10 +++++----- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql index 8a64b844043..f1fcde14ef1 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql @@ -9,15 +9,15 @@ CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickho SET insert_quorum=2; SET select_sequential_consistency=1; -INSERT INTO test.quorum2 VALUES (1, '15-10-2018'); -INSERT INTO test.quorum2 VALUES (2, '15-10-2018'); -INSERT INTO test.quorum2 VALUES (2, '15-11-2018'); +INSERT INTO test.quorum1 VALUES (1, toDate('2018-11-15')); +INSERT INTO test.quorum1 VALUES (2, toDate('2018-11-15')); +INSERT INTO test.quorum1 VALUES (1, toDate('2018-12-16')); -SET insert_quorum_timeout=100; +SET insert_quorum_timeout=0; SYSTEM STOP FETCHES test.quorum1; -INSERT INTO test.quorum2 VALUES (1, '15-12-2018'); -- { serverError 319 } -- +INSERT INTO test.quorum2 VALUES (2, toDate('2018-12-16')); -- { serverError 319 } -- SELECT count(*) FROM test.quorum1; SELECT count(*) FROM test.quorum2; @@ -28,7 +28,7 @@ SELECT count(*) FROM test.quorum2; SET select_sequential_consistency=1; -SET insert_quorum_timeout=100; +SET insert_quorum_timeout=6000; SYSTEM START FETCHES test.quorum1; SYSTEM SYNC REPLICA test.quorum1; diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql index 7a7ffa1de87..ebefa1e0505 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql @@ -13,7 +13,7 @@ SET insert_quorum_timeout=0; SYSTEM STOP FETCHES test.quorum1; -INSERT INTO test.quorum2 VALUES (1, '15-11-2018'); -- { serverError 319 } -- +INSERT INTO test.quorum2 VALUES (1, toDate('2018-11-15')); -- { serverError 319 } -- SELECT count(*) FROM test.quorum1; SELECT count(*) FROM test.quorum2; diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql index 48e0cd62be2..9ea55d25f56 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql @@ -9,8 +9,8 @@ CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickho SET insert_quorum=2; SET select_sequential_consistency=1; -INSERT INTO test.quorum1 VALUES (1, '15-11-2018'); -INSERT INTO test.quorum1 VALUES (2, '15-11-2018'); +INSERT INTO test.quorum1 VALUES (1, toDate('2018-11-15')); +INSERT INTO test.quorum1 VALUES (2, toDate('2018-11-15')); SELECT count(*) FROM test.quorum1; SELECT count(*) FROM test.quorum2; diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql index f37d1d82669..34841b73050 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql @@ -3,15 +3,15 @@ SET send_logs_level = 'none'; DROP TABLE IF EXISTS test.quorum1; DROP TABLE IF EXISTS test.quorum2; -CREATE TABLE test.quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/fetches', '1') ORDER BY x PARTITION BY y; -CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/fetches', '2') ORDER BY x PARTITION BY y; +CREATE TABLE test.quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '1') ORDER BY x PARTITION BY y; +CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '2') ORDER BY x PARTITION BY y; SET insert_quorum=2; SET select_sequential_consistency=1; -INSERT INTO test.quorum1 VALUES (1, '15-11-2018'); -INSERT INTO test.quorum1 VALUES (2, '15-11-2018'); -INSERT INTO test.quorum1 VALUES (1, '15-12-2018'); +INSERT INTO test.quorum1 VALUES (1, toDate('2018-11-15')); +INSERT INTO test.quorum1 VALUES (2, toDate('2018-11-15')); +INSERT INTO test.quorum1 VALUES (1, toDate('2018-12-16')); SELECT count(*) FROM test.quorum1; SELECT count(*) FROM test.quorum2; From 52deb3932f42b6f94f7de634693564db99ac0deb Mon Sep 17 00:00:00 2001 From: VadimPE Date: Mon, 15 Oct 2018 17:55:02 +0300 Subject: [PATCH 49/79] fix test --- ...st_part_and_alive_part_zookeeper.reference | 13 ++++++++++++ ...ert_lost_part_and_alive_part_zookeeper.sql | 20 +++++++++---------- ...0732_quorum_insert_lost_part_zookeeper.sql | 9 ++++----- ...rt_simple_test_1_parts_zookeeper.reference | 2 ++ ...m_insert_simple_test_1_parts_zookeeper.sql | 8 ++++---- ...rt_simple_test_2_parts_zookeeper.reference | 4 ++++ ...m_insert_simple_test_2_parts_zookeeper.sql | 10 +++++----- 7 files changed, 41 insertions(+), 25 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference index 336921101fa..85a224943ce 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.reference @@ -1,5 +1,18 @@ +1 +2 3 +1 +2 +3 +1 +2 3 4 +1 +2 +3 4 +1 +2 +3 4 diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql index f1fcde14ef1..a956688e893 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql @@ -9,32 +9,30 @@ CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickho SET insert_quorum=2; SET select_sequential_consistency=1; -INSERT INTO test.quorum1 VALUES (1, toDate('2018-11-15')); -INSERT INTO test.quorum1 VALUES (2, toDate('2018-11-15')); -INSERT INTO test.quorum1 VALUES (1, toDate('2018-12-16')); +INSERT INTO test.quorum1 VALUES (1, '2018-11-15'); +INSERT INTO test.quorum1 VALUES (2, '2018-11-15'); +INSERT INTO test.quorum1 VALUES (3, '2018-12-16'); SET insert_quorum_timeout=0; SYSTEM STOP FETCHES test.quorum1; -INSERT INTO test.quorum2 VALUES (2, toDate('2018-12-16')); -- { serverError 319 } -- +INSERT INTO test.quorum2 VALUES (4, toDate('2018-12-16')); -- { serverError 319 } -SELECT count(*) FROM test.quorum1; -SELECT count(*) FROM test.quorum2; +SELECT x FROM test.quorum1 ORDER BY x; +SELECT x FROM test.quorum2 ORDER BY x; SET select_sequential_consistency=0; -SELECT count(*) FROM test.quorum2; +SELECT x FROM test.quorum2 ORDER BY x; SET select_sequential_consistency=1; -SET insert_quorum_timeout=6000; - SYSTEM START FETCHES test.quorum1; SYSTEM SYNC REPLICA test.quorum1; -SELECT count(*) FROM test.quorum1; -SELECT count(*) FROM test.quorum2; +SELECT x FROM test.quorum1 ORDER BY x; +SELECT x FROM test.quorum2 ORDER BY x; DROP TABLE IF EXISTS test.quorum1; DROP TABLE IF EXISTS test.quorum2; diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql index ebefa1e0505..be333378aed 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql @@ -13,15 +13,14 @@ SET insert_quorum_timeout=0; SYSTEM STOP FETCHES test.quorum1; -INSERT INTO test.quorum2 VALUES (1, toDate('2018-11-15')); -- { serverError 319 } -- +INSERT INTO test.quorum2 VALUES (1, '2018-11-15'); -- { serverError 319 } SELECT count(*) FROM test.quorum1; SELECT count(*) FROM test.quorum2; SET select_sequential_consistency=0; -SELECT count(*) FROM test.quorum2; - +SELECT x FROM test.quorum2 ORDER BY x; SET select_sequential_consistency=1; SET insert_quorum_timeout=100; @@ -29,8 +28,8 @@ SET insert_quorum_timeout=100; SYSTEM START FETCHES test.quorum1; SYSTEM SYNC REPLICA test.quorum1; -SELECT count(*) FROM test.quorum1; -SELECT count(*) FROM test.quorum2; +SELECT x FROM test.quorum1 ORDER BY x; +SELECT x FROM test.quorum2 ORDER BY x; DROP TABLE IF EXISTS test.quorum1; DROP TABLE IF EXISTS test.quorum2; diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference index 51993f072d5..099b7d91c92 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference @@ -1,2 +1,4 @@ +1 2 +1 2 diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql index 9ea55d25f56..97969b24541 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql @@ -9,11 +9,11 @@ CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickho SET insert_quorum=2; SET select_sequential_consistency=1; -INSERT INTO test.quorum1 VALUES (1, toDate('2018-11-15')); -INSERT INTO test.quorum1 VALUES (2, toDate('2018-11-15')); +INSERT INTO test.quorum1 VALUES (1, '2018-11-15'); +INSERT INTO test.quorum1 VALUES (2, '2018-11-15'); -SELECT count(*) FROM test.quorum1; -SELECT count(*) FROM test.quorum2; +SELECT x FROM test.quorum1 ORDER BY x; +SELECT x FROM test.quorum2 ORDER BY x; DROP TABLE IF EXISTS test.quorum1; DROP TABLE IF EXISTS test.quorum2; diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.reference index a5c8806279f..2c9d316b3d6 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.reference @@ -1,2 +1,6 @@ +1 +2 3 +1 +2 3 diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql index 34841b73050..61d0d59af17 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql @@ -9,12 +9,12 @@ CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickho SET insert_quorum=2; SET select_sequential_consistency=1; -INSERT INTO test.quorum1 VALUES (1, toDate('2018-11-15')); -INSERT INTO test.quorum1 VALUES (2, toDate('2018-11-15')); -INSERT INTO test.quorum1 VALUES (1, toDate('2018-12-16')); +INSERT INTO test.quorum1 VALUES (1, '2018-11-15'); +INSERT INTO test.quorum1 VALUES (2, '2018-11-15'); +INSERT INTO test.quorum1 VALUES (3, '2018-12-16'); -SELECT count(*) FROM test.quorum1; -SELECT count(*) FROM test.quorum2; +SELECT x FROM test.quorum1 ORDER BY x; +SELECT x FROM test.quorum2 ORDER BY x; DROP TABLE IF EXISTS test.quorum1; DROP TABLE IF EXISTS test.quorum2; From 868c1bd70b72021632d30a9f73255ef5ae216b41 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Tue, 16 Oct 2018 13:52:15 +0300 Subject: [PATCH 50/79] ISSUES-2581 --- ...have_data_befor_quorum_zookeeper.reference | 18 +++++++++++ ...nsert_have_data_befor_quorum_zookeeper.sql | 30 +++++++++++++++++++ 2 files changed, 48 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_befor_quorum_zookeeper.reference create mode 100644 dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_befor_quorum_zookeeper.sql diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_befor_quorum_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_befor_quorum_zookeeper.reference new file mode 100644 index 00000000000..674c56cf9e3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_befor_quorum_zookeeper.reference @@ -0,0 +1,18 @@ +1 +2 +3 +1 +2 +3 +1 +2 +3 +4 +5 +6 +1 +2 +3 +4 +5 +6 diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_befor_quorum_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_befor_quorum_zookeeper.sql new file mode 100644 index 00000000000..1c7aa898638 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_befor_quorum_zookeeper.sql @@ -0,0 +1,30 @@ +SET send_logs_level = 'none'; + +DROP TABLE IF EXISTS test.quorum1; +DROP TABLE IF EXISTS test.quorum2; + +CREATE TABLE test.quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '1') ORDER BY x PARTITION BY y; +CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '2') ORDER BY x PARTITION BY y; + +INSERT INTO test.quorum1 VALUES (1, '1990-11-15'); +INSERT INTO test.quorum1 VALUES (2, '1990-11-15'); +INSERT INTO test.quorum1 VALUES (3, '2020-12-16'); + +SYSTEM SYNC REPLICA test.quorum2; + +SET select_sequential_consistency=1; + +SELECT x FROM test.quorum1 ORDER BY x; +SELECT x FROM test.quorum2 ORDER BY x; + +SET insert_quorum=2; + +INSERT INTO test.quorum1 VALUES (4, '1990-11-15'); +INSERT INTO test.quorum1 VALUES (5, '1990-11-15'); +INSERT INTO test.quorum1 VALUES (6, '2020-12-16'); + +SELECT x FROM test.quorum1 ORDER BY x; +SELECT x FROM test.quorum2 ORDER BY x; + +DROP TABLE IF EXISTS test.quorum1; +DROP TABLE IF EXISTS test.quorum2; From 070eae2ae6a98629b502d3dd5f078bb8e45de1bd Mon Sep 17 00:00:00 2001 From: VadimPE Date: Tue, 16 Oct 2018 14:38:49 +0300 Subject: [PATCH 51/79] fix name --- ...732_quorum_insert_have_data_before_quorum_zookeeper.reference} | 0 ... => 00732_quorum_insert_have_data_before_quorum_zookeeper.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename dbms/tests/queries/0_stateless/{00732_quorum_insert_have_data_befor_quorum_zookeeper.reference => 00732_quorum_insert_have_data_before_quorum_zookeeper.reference} (100%) rename dbms/tests/queries/0_stateless/{00732_quorum_insert_have_data_befor_quorum_zookeeper.sql => 00732_quorum_insert_have_data_before_quorum_zookeeper.sql} (100%) diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_befor_quorum_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.reference similarity index 100% rename from dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_befor_quorum_zookeeper.reference rename to dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.reference diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_befor_quorum_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql similarity index 100% rename from dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_befor_quorum_zookeeper.sql rename to dbms/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql From 133363e8e36b83e6ee7450a44d90b05b43dd8f62 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 17 Oct 2018 14:35:16 +0800 Subject: [PATCH 52/79] ISSUES-2312 fix nested view & joined view without database --- .../PredicateExpressionsOptimizer.cpp | 2 +- dbms/src/Parsers/ASTSelectQuery.cpp | 45 +++++++++++++------ .../00740_database_in_nested_view.reference | 6 +++ .../00740_database_in_nested_view.sql | 23 ++++++++++ dbms/tests/queries/bugs/database_in_view.sql | 18 -------- 5 files changed, 61 insertions(+), 33 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00740_database_in_nested_view.reference create mode 100644 dbms/tests/queries/0_stateless/00740_database_in_nested_view.sql delete mode 100644 dbms/tests/queries/bugs/database_in_view.sql diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp index 474f1454155..7d4d38f5082 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -395,7 +395,7 @@ ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_que { const auto database_and_table_ast = static_cast(table_expression->database_and_table_name.get()); const auto database_and_table_name = getDatabaseAndTableNameFromIdentifier(*database_and_table_ast); - storage = context.tryGetTable(database_and_table_name.first, database_and_table_name.second); + storage = context.getTable(database_and_table_name.first, database_and_table_name.second); } const auto block = storage->getSampleBlock(); diff --git a/dbms/src/Parsers/ASTSelectQuery.cpp b/dbms/src/Parsers/ASTSelectQuery.cpp index 5f7ea7dc91a..0cf19a28851 100644 --- a/dbms/src/Parsers/ASTSelectQuery.cpp +++ b/dbms/src/Parsers/ASTSelectQuery.cpp @@ -7,6 +7,7 @@ #include #include #include +#include "ASTSelectWithUnionQuery.h" namespace DB @@ -339,25 +340,41 @@ const ASTTablesInSelectQueryElement * ASTSelectQuery::join() const void ASTSelectQuery::setDatabaseIfNeeded(const String & database_name) { - ASTTableExpression * table_expression = getFirstTableExpression(*this); - if (!table_expression) + if (!tables) return; - if (!table_expression->database_and_table_name) - return; + ASTTablesInSelectQuery & tables_in_select_query = static_cast(*tables); - if (table_expression->database_and_table_name->children.empty()) + for (auto & child : tables_in_select_query.children) { - ASTPtr database = ASTIdentifier::createSpecial(database_name); - ASTPtr table = table_expression->database_and_table_name; + const auto & tables_element = static_cast(*child); + if (tables_element.table_expression) + { + const auto table_expression = static_cast(tables_element.table_expression.get()); - const String & old_name = static_cast(*table_expression->database_and_table_name).name; - table_expression->database_and_table_name = ASTIdentifier::createSpecial(database_name + "." + old_name); - table_expression->database_and_table_name->children = {database, table}; - } - else if (table_expression->database_and_table_name->children.size() != 2) - { - throw Exception("Logical error: more than two components in table expression", ErrorCodes::LOGICAL_ERROR); + if (!table_expression->database_and_table_name && !table_expression->subquery) + continue; + + if (table_expression->subquery) + { + const auto subquery = static_cast(table_expression->subquery.get()); + const auto select_with_union_query = static_cast(subquery->children[0].get()); + select_with_union_query->setDatabaseIfNeeded(database_name); + } + else if (table_expression->database_and_table_name->children.empty()) + { + ASTPtr database = ASTIdentifier::createSpecial(database_name); + ASTPtr table = table_expression->database_and_table_name; + + const String & old_name = static_cast(*table_expression->database_and_table_name).name; + table_expression->database_and_table_name = ASTIdentifier::createSpecial(database_name + "." + old_name); + table_expression->database_and_table_name->children = {database, table}; + } + else if (table_expression->database_and_table_name->children.size() != 2) + { + throw Exception("Logical error: more than two components in table expression", ErrorCodes::LOGICAL_ERROR); + } + } } } diff --git a/dbms/tests/queries/0_stateless/00740_database_in_nested_view.reference b/dbms/tests/queries/0_stateless/00740_database_in_nested_view.reference new file mode 100644 index 00000000000..c9854e61c3d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00740_database_in_nested_view.reference @@ -0,0 +1,6 @@ +1 +1 +1 1 +1 +1 +1 1 diff --git a/dbms/tests/queries/0_stateless/00740_database_in_nested_view.sql b/dbms/tests/queries/0_stateless/00740_database_in_nested_view.sql new file mode 100644 index 00000000000..e4dabc3a5a6 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00740_database_in_nested_view.sql @@ -0,0 +1,23 @@ +DROP TABLE IF EXISTS test.test; +DROP TABLE IF EXISTS test.test_view; +DROP TABLE IF EXISTS test.test_nested_view; +DROP TABLE IF EXISTS test.test_joined_view; + +USE test; +CREATE VIEW test AS SELECT 1 AS N; +CREATE VIEW test_view AS SELECT * FROM test; +CREATE VIEW test_nested_view AS SELECT * FROM (SELECT * FROM test); +CREATE VIEW test_joined_view AS SELECT * FROM test ANY LEFT JOIN test USING N; + +SELECT * FROM test_view; +SELECT * FROM test_nested_view; +SELECT * FROM test_joined_view; + +USE default; +SELECT * FROM test.test_view; +SELECT * FROM test.test_nested_view; +SELECT * FROM test.test_joined_view; + +DROP TABLE IF EXISTS test.test; +DROP TABLE IF EXISTS test.test_view; +DROP TABLE IF EXISTS test.test_nested_view; diff --git a/dbms/tests/queries/bugs/database_in_view.sql b/dbms/tests/queries/bugs/database_in_view.sql deleted file mode 100644 index 0b7149e4e88..00000000000 --- a/dbms/tests/queries/bugs/database_in_view.sql +++ /dev/null @@ -1,18 +0,0 @@ -DROP TABLE IF EXISTS test.whoami; -DROP TABLE IF EXISTS test.tellme; -DROP TABLE IF EXISTS test.tellme_nested; - -use test; -create view whoami as select 1 as n; -create view tellme as select * from whoami; -create view tellme_nested as select * from (select * from whoami); -select * from tellme; -select * from tellme_nested; - -use default; -select * from test.tellme; -select * from test.tellme_nested; - -DROP TABLE test.whoami; -DROP TABLE test.tellme; -DROP TABLE test.tellme_nested; From d81ed2700452968c8569ce0e3cbb7a6f5ad72a78 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Wed, 17 Oct 2018 17:08:52 +0300 Subject: [PATCH 53/79] add lock for .inner. table --- .../PushingToViewsBlockOutputStream.cpp | 5 +++++ dbms/src/Storages/StorageMaterializedView.h | 6 ++--- .../00738_lock_for_inner_table.reference | 0 .../0_stateless/00738_lock_for_inner_table.sh | 22 +++++++++++++++++++ 4 files changed, 30 insertions(+), 3 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00738_lock_for_inner_table.reference create mode 100755 dbms/tests/queries/0_stateless/00738_lock_for_inner_table.sh diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index b5178cbb17c..e7ff38ead62 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -8,6 +8,8 @@ #include #include +#include + namespace DB { @@ -44,6 +46,9 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( auto dependent_table = context.getTable(database_table.first, database_table.second); auto & materialized_view = dynamic_cast(*dependent_table); + if (StoragePtr inner_table = materialized_view.tryGetTargetTable()) + addTableLock(inner_table->lockStructure(true, __PRETTY_FUNCTION__)); + auto query = materialized_view.getInnerQuery(); BlockOutputStreamPtr out = std::make_shared( database_table.first, database_table.second, dependent_table, *views_context, ASTPtr()); diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index 33ed853f1c0..c20757b802b 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -48,6 +48,9 @@ public: QueryProcessingStage::Enum getQueryProcessingStage(const Context & context) const override; + StoragePtr getTargetTable() const; + StoragePtr tryGetTargetTable() const; + BlockInputStreams read( const Names & column_names, const SelectQueryInfo & query_info, @@ -69,9 +72,6 @@ private: Context & global_context; bool has_inner_table = false; - StoragePtr getTargetTable() const; - StoragePtr tryGetTargetTable() const; - void checkStatementCanBeForwarded() const; protected: diff --git a/dbms/tests/queries/0_stateless/00738_lock_for_inner_table.reference b/dbms/tests/queries/0_stateless/00738_lock_for_inner_table.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00738_lock_for_inner_table.sh b/dbms/tests/queries/0_stateless/00738_lock_for_inner_table.sh new file mode 100755 index 00000000000..611f1fd7564 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00738_lock_for_inner_table.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +echo "DROP TABLE IF EXISTS test.tab; +DROP TABLE IF EXISTS test.mv; +CREATE TABLE test.tab(a Int) ENGINE = Log; +CREATE MATERIALIZED VIEW test.mv ENGINE = Log AS SELECT a FROM test.tab;" | ${CLICKHOUSE_CLIENT} -n + +${CLICKHOUSE_CLIENT} --query "INSERT INTO test.tab SELECT number FROM numbers(10000000)" & + +function drop() +{ + sleep 0.1 + ${CLICKHOUSE_CLIENT} --query "DROP TABLE test.\`.inner.mv\`" -n +} + +drop & + +wait From f86b9d398304b1061ed7a80eda63eccf3cbb19c6 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Wed, 17 Oct 2018 17:11:20 +0300 Subject: [PATCH 54/79] add lock for .inner. table --- dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index e7ff38ead62..096a7b1fe2a 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -8,8 +8,6 @@ #include #include -#include - namespace DB { From 7623a3f48bc58565011131305b5e03362f1bc349 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Wed, 17 Oct 2018 19:56:00 +0300 Subject: [PATCH 55/79] fix test --- dbms/tests/queries/0_stateless/00738_lock_for_inner_table.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00738_lock_for_inner_table.sh b/dbms/tests/queries/0_stateless/00738_lock_for_inner_table.sh index 611f1fd7564..db20f6172f0 100755 --- a/dbms/tests/queries/0_stateless/00738_lock_for_inner_table.sh +++ b/dbms/tests/queries/0_stateless/00738_lock_for_inner_table.sh @@ -20,3 +20,6 @@ function drop() drop & wait + +echo "DROP TABLE IF EXISTS test.tab; +DROP TABLE IF EXISTS test.mv;" | ${CLICKHOUSE_CLIENT} -n From 4a93d3b836f5afbef18404224a6dbee2b06f460e Mon Sep 17 00:00:00 2001 From: VadimPE Date: Thu, 18 Oct 2018 14:51:40 +0300 Subject: [PATCH 56/79] fix addedparts --- .../ReplicatedMergeTreeQuorumAddedParts.h | 20 +++++-------- .../Storages/StorageReplicatedMergeTree.cpp | 30 +++++++++++-------- 2 files changed, 26 insertions(+), 24 deletions(-) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h index 0e971b42d0d..fc2673f39d2 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h @@ -21,11 +21,9 @@ struct ReplicatedMergeTreeQuorumAddedParts MergeTreeDataFormatVersion format_version; - ReplicatedMergeTreeQuorumAddedParts(const std::string & added_parts_str, MergeTreeDataFormatVersion format_version_) + ReplicatedMergeTreeQuorumAddedParts(const MergeTreeDataFormatVersion format_version_) : format_version(format_version_) - { - fromString(added_parts_str); - } + {} /// Write new parts in buffer with added parts. void write(WriteBuffer & out) @@ -43,8 +41,8 @@ struct ReplicatedMergeTreeQuorumAddedParts for (const auto & part : added_parts) { - auto partition_info = MergeTreePartInfo::fromPartName(part.second, format_version); - max_added_blocks[part.first] = partition_info.max_block; + auto part_info = MergeTreePartInfo::fromPartName(part.second, format_version); + max_added_blocks[part.first] = part_info.max_block; } return max_added_blocks; @@ -71,12 +69,12 @@ struct ReplicatedMergeTreeQuorumAddedParts { PartitionIdToPartName parts_in_quorum; - std::string partition_name; + std::string part_name; - readText(partition_name, in); + readText(part_name, in); - auto partition_info = MergeTreePartInfo::fromPartName(partition_name, format_version); - parts_in_quorum[partition_info.partition_id] = partition_name; + auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); + parts_in_quorum[part_info.partition_id] = part_name; return parts_in_quorum; } @@ -109,8 +107,6 @@ struct ReplicatedMergeTreeQuorumAddedParts void fromString(const std::string & str) { - if (str.empty()) - return; ReadBufferFromString in(str); read(in); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index bcf02abc412..613b58ae260 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2669,7 +2669,10 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) Coordination::Stat added_parts_stat; String old_added_parts = zookeeper->get(quorum_last_part_path, &added_parts_stat); - ReplicatedMergeTreeQuorumAddedParts parts_with_quorum(old_added_parts, data.format_version); + ReplicatedMergeTreeQuorumAddedParts parts_with_quorum(data.format_version); + + if (!old_added_parts.empty()) + parts_with_quorum.fromString(old_added_parts); auto partition_info = MergeTreePartInfo::fromPartName(part_name, data.format_version); parts_with_quorum.added_parts[partition_info.partition_id] = part_name; @@ -2979,20 +2982,23 @@ BlockInputStreams StorageReplicatedMergeTree::read( } String added_parts_str; - zookeeper->tryGet(zookeeper_path + "/quorum/last_part", added_parts_str); - - if (!added_parts_str.empty()) + if (zookeeper->tryGet(zookeeper_path + "/quorum/last_part", added_parts_str)) { - ReplicatedMergeTreeQuorumAddedParts part_with_quorum(added_parts_str, data.format_version); - auto added_parts = part_with_quorum.added_parts; + if (!added_parts_str.empty()) + { + ReplicatedMergeTreeQuorumAddedParts part_with_quorum(data.format_version); + part_with_quorum.fromString(added_parts_str); + + auto added_parts = part_with_quorum.added_parts; - for (const auto & added_part : added_parts) - if (!data.getActiveContainingPart(added_part.second)) - throw Exception("Replica doesn't have part " + added_part.second + " which was successfully written to quorum of other replicas." - " Send query to another replica or disable 'select_sequential_consistency' setting.", ErrorCodes::REPLICA_IS_NOT_IN_QUORUM); + for (const auto & added_part : added_parts) + if (!data.getActiveContainingPart(added_part.second)) + throw Exception("Replica doesn't have part " + added_part.second + " which was successfully written to quorum of other replicas." + " Send query to another replica or disable 'select_sequential_consistency' setting.", ErrorCodes::REPLICA_IS_NOT_IN_QUORUM); - for (const auto & max_block : part_with_quorum.getMaxInsertedBlocks()) - max_added_blocks[max_block.first] = max_block.second; + for (const auto & max_block : part_with_quorum.getMaxInsertedBlocks()) + max_added_blocks[max_block.first] = max_block.second; + } } return reader.read(column_names, query_info, context, max_block_size, num_streams, &max_added_blocks); From 4a0ca4cbc5c6b0ebc67994b5f0e1dffd115c3565 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Thu, 18 Oct 2018 15:00:36 +0300 Subject: [PATCH 57/79] ad test --- ...ata_and_without_quorum_zookeeper.reference | 6 ++++ ..._old_data_and_without_quorum_zookeeper.sql | 28 +++++++++++++++++++ 2 files changed, 34 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.reference create mode 100644 dbms/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.reference new file mode 100644 index 00000000000..2c9d316b3d6 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.reference @@ -0,0 +1,6 @@ +1 +2 +3 +1 +2 +3 diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql new file mode 100644 index 00000000000..9a21c3319de --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql @@ -0,0 +1,28 @@ +SET send_logs_level = 'none'; + +DROP TABLE IF EXISTS test.quorum1; +DROP TABLE IF EXISTS test.quorum2; + +CREATE TABLE test.quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '1') ORDER BY x PARTITION BY y; +CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '2') ORDER BY x PARTITION BY y; + +INSERT INTO test.quorum1 VALUES (1, '1990-11-15'); +INSERT INTO test.quorum1 VALUES (2, '1990-11-15'); +INSERT INTO test.quorum1 VALUES (3, '2020-12-16'); + +SYSTEM SYNC REPLICA test.quorum2; + +SET select_sequential_consistency=1; +SET insert_quorum=2; + +SET insert_quorum_timeout=0; + +SYSTEM STOP FETCHES test.quorum1; + +INSERT INTO test.quorum2 VALUES (4, toDate('2020-12-16')); -- { serverError 319 } + +SELECT x FROM test.quorum1 ORDER BY x; +SELECT x FROM test.quorum2 ORDER BY x; + +DROP TABLE IF EXISTS test.quorum1; +DROP TABLE IF EXISTS test.quorum2; From 451114c9a3a7e2780e2ac42e5d3bb4095f89fa0a Mon Sep 17 00:00:00 2001 From: VadimPE Date: Thu, 18 Oct 2018 15:16:53 +0300 Subject: [PATCH 58/79] rename partition -> part --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 613b58ae260..d72900a3d04 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2674,8 +2674,8 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) if (!old_added_parts.empty()) parts_with_quorum.fromString(old_added_parts); - auto partition_info = MergeTreePartInfo::fromPartName(part_name, data.format_version); - parts_with_quorum.added_parts[partition_info.partition_id] = part_name; + auto part_info = MergeTreePartInfo::fromPartName(part_name, data.format_version); + parts_with_quorum.added_parts[part_info.partition_id] = part_name; String new_added_parts = parts_with_quorum.toString(); @@ -2976,9 +2976,9 @@ BlockInputStreams StorageReplicatedMergeTree::read( ReplicatedMergeTreeQuorumEntry quorum_entry; quorum_entry.fromString(value); - auto partition_info = MergeTreePartInfo::fromPartName(quorum_entry.part_name, data.format_version); + auto part_info = MergeTreePartInfo::fromPartName(quorum_entry.part_name, data.format_version); - max_added_blocks[partition_info.partition_id] = partition_info.max_block - 1; + max_added_blocks[part_info.partition_id] = part_info.max_block - 1; } String added_parts_str; @@ -2988,7 +2988,7 @@ BlockInputStreams StorageReplicatedMergeTree::read( { ReplicatedMergeTreeQuorumAddedParts part_with_quorum(data.format_version); part_with_quorum.fromString(added_parts_str); - + auto added_parts = part_with_quorum.added_parts; for (const auto & added_part : added_parts) From f2db463ab0d58589ea42b73a46004ac1e5c31da1 Mon Sep 17 00:00:00 2001 From: VadimPE Date: Thu, 18 Oct 2018 17:14:07 +0300 Subject: [PATCH 59/79] fix bug with merge part, wich us inserted with quorum --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 16 ++++++++++++---- .../MergeTree/ReplicatedMergeTreeQueue.h | 3 ++- .../ReplicatedMergeTreeQuorumAddedParts.h | 1 + ...nsert_simple_test_1_parts_zookeeper.reference | 1 + ...orum_insert_simple_test_1_parts_zookeeper.sql | 4 ++++ 5 files changed, 20 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index c6c1fa27e33..69b088e4aa9 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1400,8 +1400,16 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( queue_.pullLogsToQueue(zookeeper); Coordination::GetResponse quorum_last_part_response = quorum_last_part_future.get(); - if (!quorum_last_part_response.error) - last_quorum_part = quorum_last_part_response.data; + if (!quorum_last_part_response.error) { + ReplicatedMergeTreeQuorumAddedParts parts_with_quorum(queue.format_version); + if (!quorum_last_part_response.data.empty()) + { + parts_with_quorum.fromString(quorum_last_part_response.data); + last_quorum_parts.clear(); + for (const auto & added_part : parts_with_quorum.added_parts) + last_quorum_parts.emplace(added_part.second); + } + } Coordination::GetResponse quorum_status_response = quorum_status_future.get(); if (!quorum_status_response.error) @@ -1460,7 +1468,7 @@ bool ReplicatedMergeTreeMergePredicate::operator()( for (const MergeTreeData::DataPartPtr & part : {left, right}) { - if (part->name == last_quorum_part) + if (last_quorum_parts.find(part->name) != last_quorum_parts.end()) { if (out_reason) *out_reason = "Part " + part->name + " is the most recent part with a satisfied quorum"; @@ -1563,7 +1571,7 @@ std::optional ReplicatedMergeTreeMergePredicate::getDesiredMutationVersio /// the part (checked by querying queue.virtual_parts), we can confidently assign a mutation to /// version X for this part. - if (part->name == last_quorum_part + if (last_quorum_parts.find(part->name) != last_quorum_parts.end() || part->name == inprogress_quorum_part) return {}; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 5f8196eb4f5..df9794cf232 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -375,7 +376,7 @@ private: std::unordered_map> committing_blocks; /// Quorum state taken at some later time than prev_virtual_parts. - String last_quorum_part; + std::set last_quorum_parts; String inprogress_quorum_part; }; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h index fc2673f39d2..dc251a02dcc 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h @@ -6,6 +6,7 @@ #include #include #include +#include #include diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference index 099b7d91c92..128dde7896d 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.reference @@ -2,3 +2,4 @@ 2 1 2 +2 diff --git a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql index 97969b24541..979e8932e5d 100644 --- a/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql @@ -15,5 +15,9 @@ INSERT INTO test.quorum1 VALUES (2, '2018-11-15'); SELECT x FROM test.quorum1 ORDER BY x; SELECT x FROM test.quorum2 ORDER BY x; +OPTIMIZE TABLE test.quorum1 PARTITION '2018-11-15' FINAL; + +SELECT count(*) FROM system.parts WHERE active AND database = 'test' AND table='quorum1'; + DROP TABLE IF EXISTS test.quorum1; DROP TABLE IF EXISTS test.quorum2; From a32e06d298d8fac0e82a5392506d9f5f126c45e1 Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 18 Oct 2018 18:03:14 +0300 Subject: [PATCH 60/79] more ExpressionAnalyzer refactoring [CLICKHOUSE-3996] --- dbms/src/Interpreters/ActionsVisitor.h | 5 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 29 ++-- dbms/src/Interpreters/ExpressionAnalyzer.h | 159 +++++++++++------- .../Interpreters/InterpreterCreateQuery.cpp | 16 +- .../LogicalExpressionsOptimizer.cpp | 2 +- .../LogicalExpressionsOptimizer.h | 14 +- .../PredicateExpressionsOptimizer.cpp | 2 +- .../PredicateExpressionsOptimizer.h | 23 ++- dbms/src/Interpreters/QueryNormalizer.cpp | 9 +- dbms/src/Interpreters/QueryNormalizer.h | 21 ++- .../tests/logical_expressions_optimizer.cpp | 5 +- dbms/src/Parsers/ASTAlterQuery.h | 7 +- dbms/src/Parsers/ASTCheckQuery.h | 7 +- dbms/src/Parsers/ASTCreateQuery.h | 9 +- dbms/src/Parsers/ASTDropQuery.h | 7 +- dbms/src/Parsers/ASTOptimizeQuery.h | 8 +- dbms/src/Parsers/ParserCreateQuery.cpp | 2 +- 17 files changed, 192 insertions(+), 133 deletions(-) diff --git a/dbms/src/Interpreters/ActionsVisitor.h b/dbms/src/Interpreters/ActionsVisitor.h index 41560c55c7c..b4ca04362c1 100644 --- a/dbms/src/Interpreters/ActionsVisitor.h +++ b/dbms/src/Interpreters/ActionsVisitor.h @@ -2,6 +2,8 @@ #include +#include + namespace DB { @@ -10,9 +12,6 @@ class Context; class ASTFunction; struct ProjectionManipulatorBase; -class ExpressionActions; -using ExpressionActionsPtr = std::shared_ptr; - class Set; using SetPtr = std::shared_ptr; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 013bb09e4be..564511805c0 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -1,6 +1,8 @@ #include #include +#include + #include #include #include @@ -161,12 +163,10 @@ ExpressionAnalyzer::ExpressionAnalyzer( const Names & required_result_columns_, size_t subquery_depth_, bool do_global_, - const SubqueriesForSets & subqueries_for_set_) - : query(query_), context(context_), settings(context.getSettings()), - subquery_depth(subquery_depth_), - source_columns(source_columns_), required_result_columns(required_result_columns_), - storage(storage_), - do_global(do_global_), subqueries_for_sets(subqueries_for_set_) + const SubqueriesForSets & subqueries_for_sets_) + : ExpressionAnalyzerData(source_columns_, required_result_columns_, subqueries_for_sets_), + query(query_), context(context_), settings(context.getSettings()), storage(storage_), + subquery_depth(subquery_depth_), do_global(do_global_) { select_query = typeid_cast(query.get()); @@ -210,7 +210,7 @@ ExpressionAnalyzer::ExpressionAnalyzer( InJoinSubqueriesPreprocessor(context).process(select_query); /// Optimizes logical expressions. - LogicalExpressionsOptimizer(select_query, settings).perform(); + LogicalExpressionsOptimizer(select_query, settings.min_equality_disjunction_chain_length).perform(); /// Creates a dictionary `aliases`: alias -> ASTPtr { @@ -868,7 +868,7 @@ void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_ { BlockIO res = interpretSubquery(subquery_or_table_name, context, subquery_depth + 1, {})->execute(); - SetPtr set = std::make_shared(getSetSizeLimits(settings), true); + SetPtr set = std::make_shared(settings.size_limits_for_set, true); set->setHeader(res.in->getHeader()); while (Block block = res.in->read()) @@ -925,7 +925,8 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & Block sample_block_with_calculated_columns = temp_actions->getSampleBlock(); if (sample_block_with_calculated_columns.has(args.children.at(0)->getColumnName())) - makeExplicitSet(func, sample_block_with_calculated_columns, true, context, getSetSizeLimits(settings), prepared_sets); + makeExplicitSet(func, sample_block_with_calculated_columns, true, context, + settings.size_limits_for_set, prepared_sets); } } } @@ -1048,7 +1049,7 @@ void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, bool is_conditional_tree = !isThereArrayJoin(ast) && settings.enable_conditional_computation && !only_consts; LogAST log; - ActionsVisitor actions_visitor(context, getSetSizeLimits(settings), is_conditional_tree, subquery_depth, + ActionsVisitor actions_visitor(context, settings.size_limits_for_set, is_conditional_tree, subquery_depth, source_columns, actions, prepared_sets, subqueries_for_sets, no_subqueries, only_consts, !isRemoteStorage(), log.stream()); actions_visitor.visit(ast); @@ -1062,7 +1063,7 @@ void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join, bool is_conditional_tree = !isThereArrayJoin(query) && settings.enable_conditional_computation && !only_consts; LogAST log; - ActionsVisitor actions_visitor(context, getSetSizeLimits(settings), is_conditional_tree, subquery_depth, + ActionsVisitor actions_visitor(context, settings.size_limits_for_set, is_conditional_tree, subquery_depth, source_columns, actions, prepared_sets, subqueries_for_sets, no_subqueries, only_consts, !isRemoteStorage(), log.stream()); @@ -1320,9 +1321,9 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty if (join_params.strictness == ASTTableJoin::Strictness::Unspecified && join_params.kind != ASTTableJoin::Kind::Cross) { - if (settings.join_default_strictness.toString() == "ANY") + if (settings.join_default_strictness == "ANY") join_params.strictness = ASTTableJoin::Strictness::Any; - else if (settings.join_default_strictness.toString() == "ALL") + else if (settings.join_default_strictness == "ALL") join_params.strictness = ASTTableJoin::Strictness::All; else throw Exception("Expected ANY or ALL in JOIN section, because setting (join_default_strictness) is empty", DB::ErrorCodes::EXPECTED_ALL_OR_ANY); @@ -1364,7 +1365,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty { JoinPtr join = std::make_shared( analyzed_join.key_names_left, analyzed_join.key_names_right, analyzed_join.columns_added_by_join_from_right_keys, - settings.join_use_nulls, SizeLimits(settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode), + settings.join_use_nulls, settings.size_limits_for_join, join_params.kind, join_params.strictness); /** For GLOBAL JOINs (in the case, for example, of the push method for executing GLOBAL subqueries), the following occurs diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index b0a3948def2..732295fe449 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -2,22 +2,22 @@ #include #include -#include #include -#include -#include - namespace DB { +class Block; class Context; -class ExpressionActions; struct ExpressionActionsChain; +class ExpressionActions; +using ExpressionActionsPtr = std::shared_ptr; class IAST; using ASTPtr = std::shared_ptr; +using ASTs = std::vector; +struct ASTTableJoin; class IBlockInputStream; using BlockInputStreamPtr = std::shared_ptr; @@ -31,20 +31,103 @@ class ASTExpressionList; class ASTSelectQuery; -inline SizeLimits getSetSizeLimits(const Settings & settings) +/// ExpressionAnalyzers sources, intermediates and results. It splits data and logic, allows to test them separately. +/// If you are not writing a test you probably don't need it. Use ExpressionAnalyzer itself. +struct ExpressionAnalyzerData { - return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); -} + /// Original columns. + /// First, all available columns of the table are placed here. Then (when analyzing the query), unused columns are deleted. + NamesAndTypesList source_columns; + + /// If non-empty, ignore all expressions in not from this list. + Names required_result_columns; + + SubqueriesForSets subqueries_for_sets; + PreparedSets prepared_sets; + + /// Columns after ARRAY JOIN, JOIN, and/or aggregation. + NamesAndTypesList aggregated_columns; + NamesAndTypesList array_join_columns; + + bool has_aggregation = false; + NamesAndTypesList aggregation_keys; + AggregateDescriptions aggregate_descriptions; + + bool has_global_subqueries = false; + + using Aliases = std::unordered_map; + Aliases aliases; + + /// Which column is needed to be ARRAY-JOIN'ed to get the specified. + /// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v". + NameToNameMap array_join_result_to_source; + + /// For the ARRAY JOIN section, mapping from the alias to the full column name. + /// For example, for `ARRAY JOIN [1,2] AS b` "b" -> "array(1,2)" will enter here. + NameToNameMap array_join_alias_to_name; + + /// The backward mapping for array_join_alias_to_name. + NameToNameMap array_join_name_to_alias; + + /// All new temporary tables obtained by performing the GLOBAL IN/JOIN subqueries. + Tables external_tables; + + /// Predicate optimizer overrides the sub queries + bool rewrite_subqueries = false; + +protected: + ExpressionAnalyzerData(const NamesAndTypesList & source_columns_, + const Names & required_result_columns_, + const SubqueriesForSets & subqueries_for_sets_) + : source_columns(source_columns_), + required_result_columns(required_result_columns_), + subqueries_for_sets(subqueries_for_sets_) + {} +}; /** Transforms an expression from a syntax tree into a sequence of actions to execute it. * * NOTE: if `ast` is a SELECT query from a table, the structure of this table should not change during the lifetime of ExpressionAnalyzer. */ -class ExpressionAnalyzer : private boost::noncopyable +class ExpressionAnalyzer : private ExpressionAnalyzerData, private boost::noncopyable { private: - using ExpressionActionsPtr = std::shared_ptr; + /// Extracts settings to enlight which are used (and avoid copy of others). + struct ExtractedSettings + { + const bool asterisk_left_columns_only; + const bool use_index_for_in_with_subqueries; + const bool enable_conditional_computation; + const bool join_use_nulls; + const SizeLimits size_limits_for_set; + const SizeLimits size_limits_for_join; + const String join_default_strictness; + const UInt64 min_equality_disjunction_chain_length; + + /// for PredicateExpressionsOptimizer + const bool enable_optimize_predicate_expression; + + /// for QueryNormalizer + const UInt64 max_ast_depth; + const UInt64 max_expanded_ast_elements; + const String count_distinct_implementation; + + ExtractedSettings(const Settings & settings) + : asterisk_left_columns_only(settings.asterisk_left_columns_only), + use_index_for_in_with_subqueries(settings.use_index_for_in_with_subqueries), + enable_conditional_computation(settings.enable_conditional_computation), + join_use_nulls(settings.join_use_nulls), + size_limits_for_set(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode), + size_limits_for_join(settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode), + join_default_strictness(settings.join_default_strictness.toString()), + min_equality_disjunction_chain_length(settings.optimize_min_equality_disjunction_chain_length), + enable_optimize_predicate_expression(settings.enable_optimize_predicate_expression), + max_ast_depth(settings.max_ast_depth), + max_expanded_ast_elements(settings.max_expanded_ast_elements), + count_distinct_implementation(settings.count_distinct_implementation) + {} + }; public: ExpressionAnalyzer( @@ -137,37 +220,10 @@ private: ASTPtr query; ASTSelectQuery * select_query; const Context & context; - const Settings settings; + const ExtractedSettings settings; + StoragePtr storage; /// The main table in FROM clause, if exists. size_t subquery_depth; - - /** Original columns. - * First, all available columns of the table are placed here. Then (when analyzing the query), unused columns are deleted. - */ - NamesAndTypesList source_columns; - - /** If non-empty, ignore all expressions in not from this list. - */ - Names required_result_columns; - - /// Columns after ARRAY JOIN, JOIN, and/or aggregation. - NamesAndTypesList aggregated_columns; - - NamesAndTypesList array_join_columns; - - /// The main table in FROM clause, if exists. - StoragePtr storage; - - bool has_aggregation = false; - NamesAndTypesList aggregation_keys; - AggregateDescriptions aggregate_descriptions; - - /// Do I need to prepare for execution global subqueries when analyzing the query. - bool do_global; - bool has_global_subqueries = false; - - SubqueriesForSets subqueries_for_sets; - - PreparedSets prepared_sets; + bool do_global; /// Do I need to prepare for execution global subqueries when analyzing the query. struct AnalyzedJoin { @@ -228,29 +284,6 @@ private: AnalyzedJoin analyzed_join; - using Aliases = std::unordered_map; - Aliases aliases; - - using SetOfASTs = std::set; - using MapOfASTs = std::map; - - /// Which column is needed to be ARRAY-JOIN'ed to get the specified. - /// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v". - NameToNameMap array_join_result_to_source; - - /// For the ARRAY JOIN section, mapping from the alias to the full column name. - /// For example, for `ARRAY JOIN [1,2] AS b` "b" -> "array(1,2)" will enter here. - NameToNameMap array_join_alias_to_name; - - /// The backward mapping for array_join_alias_to_name. - NameToNameMap array_join_name_to_alias; - - - /// All new temporary tables obtained by performing the GLOBAL IN/JOIN subqueries. - Tables external_tables; - - /// Predicate optimizer overrides the sub queries - bool rewrite_subqueries = false; /** Remove all unnecessary columns from the list of all available columns of the table (`columns`). * At the same time, form a set of unknown columns (`unknown_required_source_columns`), diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 69e7ae63a15..c1c4c6f0706 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -443,7 +443,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const { if (create.storage) { - if (create.is_temporary && create.storage->engine->name != "Memory") + if (create.temporary && create.storage->engine->name != "Memory") throw Exception( "Temporary tables can only be created with ENGINE = Memory, not " + create.storage->engine->name, ErrorCodes::INCORRECT_QUERY); @@ -451,7 +451,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const return; } - if (create.is_temporary) + if (create.temporary) { auto engine_ast = std::make_shared(); engine_ast->name = "Memory"; @@ -546,7 +546,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) String data_path; DatabasePtr database; - if (!create.is_temporary) + if (!create.temporary) { database = context.getDatabase(database_name); data_path = database->getDataPath(); @@ -578,7 +578,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) create.attach, false); - if (create.is_temporary) + if (create.temporary) context.getSessionContext().addExternalTable(table_name, res, query_ptr); else database->createTable(context, table_name, res, query_ptr); @@ -601,17 +601,17 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) { auto insert = std::make_shared(); - if (!create.is_temporary) + if (!create.temporary) insert->database = database_name; insert->table = table_name; insert->select = create.select->clone(); - if (create.is_temporary && !context.getSessionContext().hasQueryContext()) + if (create.temporary && !context.getSessionContext().hasQueryContext()) context.getSessionContext().setQueryContext(context.getSessionContext()); return InterpreterInsertQuery(insert, - create.is_temporary ? context.getSessionContext() : context, + create.temporary ? context.getSessionContext() : context, context.getSettingsRef().insert_allow_materialized_columns).execute(); } @@ -657,7 +657,7 @@ void InterpreterCreateQuery::checkAccess(const ASTCreateQuery & create) throw Exception("Cannot create database. DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED); } - if (create.is_temporary && readonly >= 2) + if (create.temporary && readonly >= 2) return; if (readonly) diff --git a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp index b9aa9107afe..9705b19862b 100644 --- a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp @@ -30,7 +30,7 @@ bool LogicalExpressionsOptimizer::OrWithExpression::operator<(const OrWithExpres return std::tie(this->or_function, this->expression) < std::tie(rhs.or_function, rhs.expression); } -LogicalExpressionsOptimizer::LogicalExpressionsOptimizer(ASTSelectQuery * select_query_, const Settings & settings_) +LogicalExpressionsOptimizer::LogicalExpressionsOptimizer(ASTSelectQuery * select_query_, LogicalExpressionsOptimizerSettings && settings_) : select_query(select_query_), settings(settings_) { } diff --git a/dbms/src/Interpreters/LogicalExpressionsOptimizer.h b/dbms/src/Interpreters/LogicalExpressionsOptimizer.h index 8b0e1c7f9c5..a559ec5588a 100644 --- a/dbms/src/Interpreters/LogicalExpressionsOptimizer.h +++ b/dbms/src/Interpreters/LogicalExpressionsOptimizer.h @@ -16,6 +16,16 @@ struct Settings; class ASTFunction; class ASTSelectQuery; +struct LogicalExpressionsOptimizerSettings +{ + const UInt64 optimize_min_equality_disjunction_chain_length; + + LogicalExpressionsOptimizerSettings(UInt64 optimize_min_equality_disjunction_chain_length_) + : optimize_min_equality_disjunction_chain_length(optimize_min_equality_disjunction_chain_length_) + {} +}; + + /** This class provides functions for optimizing boolean expressions within queries. * * For simplicity, we call a homogeneous OR-chain any expression having the following structure: @@ -26,7 +36,7 @@ class LogicalExpressionsOptimizer final { public: /// Constructor. Accepts the root of the query DAG. - LogicalExpressionsOptimizer(ASTSelectQuery * select_query_, const Settings & settings_); + LogicalExpressionsOptimizer(ASTSelectQuery * select_query_, LogicalExpressionsOptimizerSettings && settings_); /** Replace all rather long homogeneous OR-chains expr = x1 OR ... OR expr = xN * on the expressions `expr` IN (x1, ..., xN). @@ -90,7 +100,7 @@ private: private: ASTSelectQuery * select_query; - const Settings & settings; + const LogicalExpressionsOptimizerSettings settings; /// Information about the OR-chains inside the query. DisjunctiveEqualityChainsMap disjunctive_equality_chains_map; /// Number of processed OR-chains. diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp index 474f1454155..22127fbdd3f 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -16,7 +16,7 @@ namespace DB static constexpr auto and_function_name = "and"; PredicateExpressionsOptimizer::PredicateExpressionsOptimizer( - ASTSelectQuery * ast_select_, const Settings & settings_, const Context & context_) + ASTSelectQuery * ast_select_, ExtractedSettings && settings_, const Context & context_) : ast_select(ast_select_), settings(settings_), context(context_) { } diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h index 6687860d979..298a4b581c8 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h @@ -41,14 +41,33 @@ using IdentifiersWithQualifiedNameSet = std::vector */ class PredicateExpressionsOptimizer { + /// Extracts settings, mostly to show which are used and which are not. + struct ExtractedSettings + { + const bool enable_optimize_predicate_expression; + + /// QueryNormalizer settings + const UInt64 max_ast_depth; + const UInt64 max_expanded_ast_elements; + const String count_distinct_implementation; + + template + ExtractedSettings(const T & settings) + : enable_optimize_predicate_expression(settings.enable_optimize_predicate_expression), + max_ast_depth(settings.max_ast_depth), + max_expanded_ast_elements(settings.max_expanded_ast_elements), + count_distinct_implementation(settings.count_distinct_implementation) + {} + }; + public: - PredicateExpressionsOptimizer(ASTSelectQuery * ast_select_, const Settings & settings_, const Context & context_); + PredicateExpressionsOptimizer(ASTSelectQuery * ast_select_, ExtractedSettings && settings_, const Context & context_); bool optimize(); private: ASTSelectQuery * ast_select; - const Settings & settings; + const ExtractedSettings settings; const Context & context; enum OptimizeKind diff --git a/dbms/src/Interpreters/QueryNormalizer.cpp b/dbms/src/Interpreters/QueryNormalizer.cpp index a1c6997368c..4aa2db7ad7f 100644 --- a/dbms/src/Interpreters/QueryNormalizer.cpp +++ b/dbms/src/Interpreters/QueryNormalizer.cpp @@ -9,7 +9,8 @@ #include #include #include -#include +//#include +#include namespace DB { @@ -22,9 +23,9 @@ namespace ErrorCodes QueryNormalizer::QueryNormalizer(ASTPtr & query, const QueryNormalizer::Aliases & aliases, - const Settings & settings, const Names & all_column_names, + ExtractedSettings && settings_, const Names & all_column_names, const TableNamesAndColumnNames & table_names_and_column_names) - : query(query), aliases(aliases), settings(settings), all_column_names(all_column_names), + : query(query), aliases(aliases), settings(settings_), all_column_names(all_column_names), table_names_and_column_names(table_names_and_column_names) { } @@ -52,7 +53,7 @@ void QueryNormalizer::perform() void QueryNormalizer::performImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOfASTs & current_asts, std::string current_alias, size_t level) { if (level > settings.max_ast_depth) - throw Exception("Normalized AST is too deep. Maximum: " + settings.max_ast_depth.toString(), ErrorCodes::TOO_DEEP_AST); + throw Exception("Normalized AST is too deep. Maximum: " + toString(settings.max_ast_depth), ErrorCodes::TOO_DEEP_AST); if (finished_asts.count(ast)) { diff --git a/dbms/src/Interpreters/QueryNormalizer.h b/dbms/src/Interpreters/QueryNormalizer.h index f5192dadd52..fd0d8603b3e 100644 --- a/dbms/src/Interpreters/QueryNormalizer.h +++ b/dbms/src/Interpreters/QueryNormalizer.h @@ -2,7 +2,6 @@ #include #include -#include #include namespace DB @@ -22,12 +21,28 @@ inline bool functionIsInOrGlobalInOperator(const String & name) using TableNameAndColumnNames = std::pair; using TableNamesAndColumnNames = std::vector; + class QueryNormalizer { + /// Extracts settings, mostly to show which are used and which are not. + struct ExtractedSettings + { + const UInt64 max_ast_depth; + const UInt64 max_expanded_ast_elements; + const String count_distinct_implementation; + + template + ExtractedSettings(const T & settings) + : max_ast_depth(settings.max_ast_depth), + max_expanded_ast_elements(settings.max_expanded_ast_elements), + count_distinct_implementation(settings.count_distinct_implementation) + {} + }; + public: using Aliases = std::unordered_map; - QueryNormalizer(ASTPtr & query, const Aliases & aliases, const Settings & settings, const Names & all_columns_name, + QueryNormalizer(ASTPtr & query, const Aliases & aliases, ExtractedSettings && settings, const Names & all_columns_name, const TableNamesAndColumnNames & table_names_and_column_names); void perform(); @@ -38,7 +53,7 @@ private: ASTPtr & query; const Aliases & aliases; - const Settings & settings; + const ExtractedSettings settings; const Names & all_column_names; const TableNamesAndColumnNames & table_names_and_column_names; diff --git a/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp b/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp index 2ae15fbdae3..f9ba76a667e 100644 --- a/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp +++ b/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp @@ -213,10 +213,7 @@ TestResult check(const TestEntry & entry) auto select_query = typeid_cast(&*ast_input); - DB::Settings settings; - settings.optimize_min_equality_disjunction_chain_length = entry.limit; - - DB::LogicalExpressionsOptimizer optimizer(select_query, settings); + DB::LogicalExpressionsOptimizer optimizer(select_query, entry.limit); optimizer.perform(); /// Parse the expected result. diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index c79f9ba8b2f..ced92bfb9bd 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include @@ -113,12 +113,9 @@ protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; }; -class ASTAlterQuery : public ASTQueryWithOutput, public ASTQueryWithOnCluster +class ASTAlterQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCluster { public: - String database; - String table; - ASTAlterCommandList * command_list = nullptr; String getID() const override; diff --git a/dbms/src/Parsers/ASTCheckQuery.h b/dbms/src/Parsers/ASTCheckQuery.h index 14274847592..a87f68c855b 100644 --- a/dbms/src/Parsers/ASTCheckQuery.h +++ b/dbms/src/Parsers/ASTCheckQuery.h @@ -1,11 +1,11 @@ #pragma once -#include +#include namespace DB { -struct ASTCheckQuery : public ASTQueryWithOutput +struct ASTCheckQuery : public ASTQueryWithTableAndOutput { /** Get the text that identifies this element. */ String getID() const override { return ("CheckQuery_" + database + "_" + table); } @@ -18,9 +18,6 @@ struct ASTCheckQuery : public ASTQueryWithOutput return res; } - std::string database; - std::string table; - protected: void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked frame) const override { diff --git a/dbms/src/Parsers/ASTCreateQuery.h b/dbms/src/Parsers/ASTCreateQuery.h index f257f6686d0..100aee8e4f0 100644 --- a/dbms/src/Parsers/ASTCreateQuery.h +++ b/dbms/src/Parsers/ASTCreateQuery.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include @@ -74,7 +74,7 @@ public: /// CREATE TABLE or ATTACH TABLE query -class ASTCreateQuery : public ASTQueryWithOutput, public ASTQueryWithOnCluster +class ASTCreateQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCluster { public: bool attach{false}; /// Query ATTACH TABLE, not CREATE TABLE. @@ -82,9 +82,6 @@ public: bool is_view{false}; bool is_materialized_view{false}; bool is_populate{false}; - bool is_temporary{false}; - String database; - String table; ASTExpressionList * columns = nullptr; String to_database; /// For CREATE MATERIALIZED VIEW mv TO table. String to_table; @@ -155,7 +152,7 @@ protected: settings.ostr << (settings.hilite ? hilite_keyword : "") << (attach ? "ATTACH " : "CREATE ") - << (is_temporary ? "TEMPORARY " : "") + << (temporary ? "TEMPORARY " : "") << what << " " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "") diff --git a/dbms/src/Parsers/ASTDropQuery.h b/dbms/src/Parsers/ASTDropQuery.h index 64d30b9453f..0916eb757a0 100644 --- a/dbms/src/Parsers/ASTDropQuery.h +++ b/dbms/src/Parsers/ASTDropQuery.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -9,7 +9,7 @@ namespace DB /** DROP query */ -class ASTDropQuery : public ASTQueryWithOutput, public ASTQueryWithOnCluster +class ASTDropQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCluster { public: enum Kind @@ -21,9 +21,6 @@ public: Kind kind; bool if_exists{false}; - bool temporary{false}; - String database; - String table; /** Get the text that identifies this element. */ String getID() const override; diff --git a/dbms/src/Parsers/ASTOptimizeQuery.h b/dbms/src/Parsers/ASTOptimizeQuery.h index 0b329d59559..269ea5b19ff 100644 --- a/dbms/src/Parsers/ASTOptimizeQuery.h +++ b/dbms/src/Parsers/ASTOptimizeQuery.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include namespace DB @@ -10,12 +10,9 @@ namespace DB /** OPTIMIZE query */ -class ASTOptimizeQuery : public ASTQueryWithOutput, public ASTQueryWithOnCluster +class ASTOptimizeQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCluster { public: - String database; - String table; - /// The partition to optimize can be specified. ASTPtr partition; /// A flag can be specified - perform optimization "to the end" instead of one step. @@ -44,7 +41,6 @@ public: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; ASTPtr getRewrittenASTWithoutOnCluster(const std::string &new_database) const override; - }; } diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index 6eaf5c4d5d8..9b2a2fc931c 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -391,7 +391,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) query->is_view = is_view; query->is_materialized_view = is_materialized_view; query->is_populate = is_populate; - query->is_temporary = is_temporary; + query->temporary = is_temporary; if (database) query->database = typeid_cast(*database).name; From 37e03bd938f906d274ef147e7c55d2bf16f7c034 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marek=20Vavru=C5=A1a?= Date: Tue, 16 Oct 2018 11:21:27 -0700 Subject: [PATCH 61/79] StorageKafka: check dependencies recursively, add max_block setting Changes: * Check table dependencies recursively before starting streaming * Use actual number of created consumers when streaming * Add size limits hint to Kafka consumer stream * Configurable `kafka_max_block_size` per table --- dbms/src/Storages/Kafka/KafkaSettings.h | 3 +- dbms/src/Storages/Kafka/StorageKafka.cpp | 90 +++++++++++++++------ dbms/src/Storages/Kafka/StorageKafka.h | 8 +- dbms/src/Storages/StorageMaterializedView.h | 2 +- 4 files changed, 75 insertions(+), 28 deletions(-) diff --git a/dbms/src/Storages/Kafka/KafkaSettings.h b/dbms/src/Storages/Kafka/KafkaSettings.h index bd7a5cc0bbb..cde3221df45 100644 --- a/dbms/src/Storages/Kafka/KafkaSettings.h +++ b/dbms/src/Storages/Kafka/KafkaSettings.h @@ -26,7 +26,8 @@ struct KafkaSettings M(SettingString, kafka_format, "", "Message format for Kafka engine.") \ M(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.") \ M(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine") \ - M(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.") + M(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.") \ + M(SettingUInt64, kafka_max_block_size, 0, "The maximum block size per table for Kafka engine.") #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ TYPE NAME {DEFAULT}; diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 5b014aa7511..fb55db8bbb9 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -268,7 +269,8 @@ StorageKafka::StorageKafka( Context & context_, const ColumnsDescription & columns_, const String & brokers_, const String & group_, const Names & topics_, - const String & format_name_, char row_delimiter_, const String & schema_name_, size_t num_consumers_) + const String & format_name_, char row_delimiter_, const String & schema_name_, + size_t num_consumers_, size_t max_block_size_) : IStorage{columns_}, table_name(table_name_), database_name(database_name_), context(context_), topics(context.getMacros()->expand(topics_)), @@ -277,7 +279,7 @@ StorageKafka::StorageKafka( format_name(context.getMacros()->expand(format_name_)), row_delimiter(row_delimiter_), schema_name(context.getMacros()->expand(schema_name_)), - num_consumers(num_consumers_), log(&Logger::get("StorageKafka (" + table_name_ + ")")), + num_consumers(num_consumers_), max_block_size(max_block_size_), log(&Logger::get("StorageKafka (" + table_name_ + ")")), semaphore(0, num_consumers_), mutex(), consumers() { task = context.getSchedulePool().createTask(log->name(), [this]{ streamThread(); }); @@ -295,10 +297,10 @@ BlockInputStreams StorageKafka::read( { check(column_names); - if (num_consumers == 0) + if (num_created_consumers == 0) return BlockInputStreams(); - const size_t stream_count = std::min(num_streams, num_consumers); + const size_t stream_count = std::min(num_streams, num_created_consumers); BlockInputStreams streams; streams.reserve(stream_count); @@ -434,26 +436,44 @@ void StorageKafka::pushConsumer(StorageKafka::ConsumerPtr c) semaphore.set(); } +bool StorageKafka::checkDependencies(const String & database_name, const String & table_name) +{ + // Check if all dependencies are attached + auto dependencies = context.getDependencies(database_name, table_name); + if (dependencies.size() == 0) + return true; + + // Check the dependencies are ready? + for (const auto & db_tab : dependencies) + { + auto table = context.tryGetTable(db_tab.first, db_tab.second); + if (!table) + return false; + + // If it materialized view, check it's target table + auto * materialized_view = dynamic_cast(table.get()); + if (materialized_view && !materialized_view->tryGetTargetTable()) + return false; + + // Check all its dependencies + if (!checkDependencies(db_tab.first, db_tab.second)) + return false; + } + + return true; +} + void StorageKafka::streamThread() { try { - // Keep streaming as long as there are attached views and streaming is not cancelled - while (!stream_cancelled) - { - // Check if all dependencies are attached - auto dependencies = context.getDependencies(database_name, table_name); - if (dependencies.size() == 0) - break; + // Check if at least one direct dependency is attached + auto dependencies = context.getDependencies(database_name, table_name); - // Check the dependencies are ready? - bool ready = true; - for (const auto & db_tab : dependencies) - { - if (!context.tryGetTable(db_tab.first, db_tab.second)) - ready = false; - } - if (!ready) + // Keep streaming as long as there are attached views and streaming is not cancelled + while (!stream_cancelled && num_created_consumers > 0 && dependencies.size() > 0) + { + if (!checkDependencies(database_name, table_name)) break; LOG_DEBUG(log, "Started streaming to " << dependencies.size() << " attached views"); @@ -488,12 +508,14 @@ bool StorageKafka::streamToViews() // Limit the number of batched messages to allow early cancellations const Settings & settings = context.getSettingsRef(); - const size_t block_size = settings.max_block_size.value; + size_t block_size = max_block_size; + if (block_size == 0) + block_size = settings.max_block_size.value; // Create a stream for each consumer and join them in a union stream BlockInputStreams streams; - streams.reserve(num_consumers); - for (size_t i = 0; i < num_consumers; ++i) + streams.reserve(num_created_consumers); + for (size_t i = 0; i < num_created_consumers; ++i) { auto stream = std::make_shared(*this, context, schema_name, block_size); streams.emplace_back(stream); @@ -509,7 +531,7 @@ bool StorageKafka::streamToViews() // Join multiple streams if necessary BlockInputStreamPtr in; if (streams.size() > 1) - in = std::make_shared>(streams, nullptr, num_consumers); + in = std::make_shared>(streams, nullptr, streams.size()); else in = streams[0]; @@ -644,6 +666,7 @@ void registerStorageKafka(StorageFactory & factory) CHECK_KAFKA_STORAGE_ARGUMENT(5, kafka_row_delimiter) CHECK_KAFKA_STORAGE_ARGUMENT(6, kafka_schema) CHECK_KAFKA_STORAGE_ARGUMENT(7, kafka_num_consumers) + CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size) #undef CHECK_KAFKA_STORAGE_ARGUMENT // Get and check broker list @@ -790,9 +813,28 @@ void registerStorageKafka(StorageFactory & factory) num_consumers = kafka_settings.kafka_num_consumers.value; } + // Parse max block size (optional) + size_t max_block_size = 0; + if (args_count >= 8) + { + auto ast = typeid_cast(engine_args[7].get()); + if (ast && ast->value.getType() == Field::Types::UInt64) + { + max_block_size = static_cast(safeGet(ast->value)); + } + else + { + throw Exception("Maximum block size must be a positive integer", ErrorCodes::BAD_ARGUMENTS); + } + } + else if (kafka_settings.kafka_max_block_size.changed) + { + max_block_size = static_cast(kafka_settings.kafka_max_block_size.value); + } + return StorageKafka::create( args.table_name, args.database_name, args.context, args.columns, - brokers, group, topics, format, row_delimiter, schema, num_consumers); + brokers, group, topics, format, row_delimiter, schema, num_consumers, max_block_size); }); } diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index f7e77b77c7a..0a1162be2c0 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -81,8 +81,10 @@ private: // in order to make various input stream parsers happy. char row_delimiter; const String schema_name; - /// Total number of consumers + /// Total number of consumers size_t num_consumers; + /// Maximum block size for insertion into this table + size_t max_block_size; /// Number of actually created consumers. /// Can differ from num_consumers in case of exception in startup() (or if startup() hasn't been called). /// In this case we still need to be able to shutdown() properly. @@ -105,6 +107,7 @@ private: void streamThread(); bool streamToViews(); + bool checkDependencies(const String & database_name, const String & table_name); protected: StorageKafka( @@ -113,7 +116,8 @@ protected: Context & context_, const ColumnsDescription & columns_, const String & brokers_, const String & group_, const Names & topics_, - const String & format_name_, char row_delimiter_, const String & schema_name_, size_t num_consumers_); + const String & format_name_, char row_delimiter_, const String & schema_name_, + size_t num_consumers_, size_t max_block_size_); }; } diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index 33ed853f1c0..52f27893782 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -57,6 +57,7 @@ public: unsigned num_streams) override; String getDataPath() const override; + StoragePtr tryGetTargetTable() const; private: String select_database_name; @@ -70,7 +71,6 @@ private: bool has_inner_table = false; StoragePtr getTargetTable() const; - StoragePtr tryGetTargetTable() const; void checkStatementCanBeForwarded() const; From 8a72ddd94b47384bc2263490e2e0d2c6ca99213c Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 19 Oct 2018 15:02:31 +0300 Subject: [PATCH 62/79] wip on finish sorting --- dbms/src/Core/SortDescription.h | 11 ++ .../FinishSortingBlockInputStream.cpp | 164 ++++++++++++++++ .../FinishSortingBlockInputStream.h | 51 +++++ .../MergeSortingBlockInputStream.cpp | 184 +----------------- .../MergeSortingBlockInputStream.h | 42 ---- dbms/src/DataStreams/processConstants.cpp | 50 +++++ dbms/src/DataStreams/processConstants.h | 23 +++ .../tests/finish_sorting_stream.cpp | 3 +- 8 files changed, 302 insertions(+), 226 deletions(-) create mode 100644 dbms/src/DataStreams/FinishSortingBlockInputStream.cpp create mode 100644 dbms/src/DataStreams/FinishSortingBlockInputStream.h create mode 100644 dbms/src/DataStreams/processConstants.cpp create mode 100644 dbms/src/DataStreams/processConstants.h diff --git a/dbms/src/Core/SortDescription.h b/dbms/src/Core/SortDescription.h index e93bfe258c5..e0d3da301db 100644 --- a/dbms/src/Core/SortDescription.h +++ b/dbms/src/Core/SortDescription.h @@ -26,6 +26,17 @@ struct SortColumnDescription SortColumnDescription(const std::string & column_name_, int direction_, int nulls_direction_, const std::shared_ptr & collator_ = nullptr) : column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) {} + + bool operator == (const SortColumnDescription & other) const + { + return column_name == other.column_name && column_number == other.column_number + && direction == other.direction && nulls_direction == other.nulls_direction; + } + + bool operator != (const SortColumnDescription & other) const + { + return !(*this == other); + } }; /// Description of the sorting rule for several columns. diff --git a/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp b/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp new file mode 100644 index 00000000000..67ea04fc2e5 --- /dev/null +++ b/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp @@ -0,0 +1,164 @@ +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +static bool isPrefix(const SortDescription & pref_descr, const SortDescription & descr) +{ + if (pref_descr.size() > descr.size()) + return false; + + for (size_t i = 0; i < pref_descr.size(); ++i) + if (pref_descr[i] != descr[i]) + return false; + + return true; +} + +FinishSortingBlockInputStream::FinishSortingBlockInputStream( + const BlockInputStreamPtr & input, const SortDescription & description_sorted_, + const SortDescription & description_to_sort_, + size_t max_merged_block_size_, size_t limit_) + : description_sorted(description_sorted_), description_to_sort(description_to_sort_), + max_merged_block_size(max_merged_block_size_), limit(limit_) + +{ + if (!isPrefix(description_sorted, description_to_sort)) + throw Exception("Can`t finish sorting. SortDescription of already sorted streamis not prefix of SortDescription needed to sort", ErrorCodes::LOGICAL_ERROR); + + children.push_back(input); + header = children.at(0)->getHeader(); + removeConstantsFromSortDescription(header, description_to_sort); +} + + +struct Less +{ + const ColumnsWithSortDescriptions & left_columns; + const ColumnsWithSortDescriptions & right_columns; + + Less(const ColumnsWithSortDescriptions & left_columns_, const ColumnsWithSortDescriptions & right_columns_) : + left_columns(left_columns_), right_columns(right_columns_) {} + + bool operator() (size_t a, size_t b) const + { + for (auto it = left_columns.begin(), jt = right_columns.begin(); it != left_columns.end(); ++it, ++jt) + { + int res = it->second.direction * it->first->compareAt(a, b, *jt->first, it->second.nulls_direction); + if (res < 0) + return true; + else if (res > 0) + return false; + } + return false; + } +}; + +Block FinishSortingBlockInputStream::readImpl() +{ + if (limit && total_rows_processed == limit) + return {}; + + Block res; + if (impl) + res = impl->read(); + + /// If res block is empty, we have finished sorting previous chunk of blocks. + if (!res) + { + if (end_of_stream) + return {}; + + blocks.clear(); + if (tail_block) + blocks.push_back(std::move(tail_block)); + + Block block; + size_t tail_pos = 0; + while (true) + { + block = children.back()->read(); + + /// End of input stream, but we can`t return immediatly, we need to merge already read blocks. + /// Check it later, when get end of stream from impl. + if (!block) + { + end_of_stream = true; + break; + } + + // If there were only const columns in sort description, then there is no need to sort. + // Return the blocks as is. + if (description_to_sort.empty()) + return block; + + size_t size = block.rows(); + if (size == 0) + continue; + + /// We need to sort each block separatly before merging. + sortBlock(block, description_to_sort); + + removeConstantsFromBlock(block); + + /// Find the position of last already read key in current block. + if (!blocks.empty()) + { + const Block & last_block = blocks.back(); + auto last_columns = getColumnsWithSortDescription(last_block, description_sorted); + auto current_columns = getColumnsWithSortDescription(block, description_sorted); + + Less less(last_columns, current_columns); + + IColumn::Permutation perm(size); + for (size_t i = 0; i < size; ++i) + perm[i] = i; + + auto it = std::upper_bound(perm.begin(), perm.end(), last_block.rows() - 1, less); + + /// We need to save tail of block, because next block may starts with the same key as in tail + /// and we should sort these rows in one chunk. + if (it != perm.end()) + { + tail_pos = it - perm.begin(); + Block head_block = block.cloneEmpty(); + tail_block = block.cloneEmpty(); + + for (size_t i = 0; i < block.columns(); ++i) + { + head_block.getByPosition(i).column = block.getByPosition(i).column->cut(0, tail_pos); + tail_block.getByPosition(i).column = block.getByPosition(i).column->cut(tail_pos, block.rows() - tail_pos); + } + + if (head_block.rows()) + blocks.push_back(head_block); + + break; + } + } + + /// If we reach here, that means that current block is first in chunk + /// or it all consists of rows with the same key as tail of a previous block. + blocks.push_back(block); + } + + impl = std::make_unique(blocks, description_to_sort, max_merged_block_size, limit); + res = impl->read(); + } + + if (res) + enrichBlockWithConstants(res, header); + + total_rows_processed += res.rows(); + + return res; +} +} \ No newline at end of file diff --git a/dbms/src/DataStreams/FinishSortingBlockInputStream.h b/dbms/src/DataStreams/FinishSortingBlockInputStream.h new file mode 100644 index 00000000000..5afaa5f1512 --- /dev/null +++ b/dbms/src/DataStreams/FinishSortingBlockInputStream.h @@ -0,0 +1,51 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +/** Takes stream already sorted by `x` and finishes sorting it by (`x`, `y`). + * During sorting only blocks with rows that equal by `x` saved in RAM. + * */ +class FinishSortingBlockInputStream : public IProfilingBlockInputStream +{ +public: + /// limit - if not 0, allowed to return just first 'limit' rows in sorted order. + FinishSortingBlockInputStream(const BlockInputStreamPtr & input, const SortDescription & description_sorted_, + const SortDescription & description_to_sort_, + size_t max_merged_block_size_, size_t limit_); + + String getName() const override { return "FinishSorting"; } + + bool isSortedOutput() const override { return true; } + const SortDescription & getSortDescription() const override { return description_to_sort; } + + Block getHeader() const override { return header; } + +protected: + Block readImpl() override; + +private: + SortDescription description_sorted; + SortDescription description_to_sort; + size_t max_merged_block_size; + size_t limit; + + Block tail_block; + Blocks blocks; + + std::unique_ptr impl; + + /// Before operation, will remove constant columns from blocks. And after, place constant columns back. + /// to avoid excessive virtual function calls + /// Save original block structure here. + Block header; + + bool end_of_stream = false; + size_t total_rows_processed = 0; +}; +} \ No newline at end of file diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp index 999b554fb0f..527adc4ec64 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -17,54 +18,6 @@ namespace ProfileEvents namespace DB { - -/** Remove constant columns from block. - */ -static void removeConstantsFromBlock(Block & block) -{ - size_t columns = block.columns(); - size_t i = 0; - while (i < columns) - { - if (block.getByPosition(i).column->isColumnConst()) - { - block.erase(i); - --columns; - } - else - ++i; - } -} - -static void removeConstantsFromSortDescription(const Block & header, SortDescription & description) -{ - description.erase(std::remove_if(description.begin(), description.end(), - [&](const SortColumnDescription & elem) - { - if (!elem.column_name.empty()) - return header.getByName(elem.column_name).column->isColumnConst(); - else - return header.safeGetByPosition(elem.column_number).column->isColumnConst(); - }), description.end()); -} - -/** Add into block, whose constant columns was removed by previous function, - * constant columns from header (which must have structure as before removal of constants from block). - */ -static void enrichBlockWithConstants(Block & block, const Block & header) -{ - size_t rows = block.rows(); - size_t columns = header.columns(); - - for (size_t i = 0; i < columns; ++i) - { - const auto & col_type_name = header.getByPosition(i); - if (col_type_name.column->isColumnConst()) - block.insert(i, {col_type_name.column->cloneResized(rows), col_type_name.type, col_type_name.name}); - } -} - - MergeSortingBlockInputStream::MergeSortingBlockInputStream( const BlockInputStreamPtr & input, SortDescription & description_, size_t max_merged_block_size_, size_t limit_, size_t max_bytes_before_remerge_, @@ -304,139 +257,4 @@ void MergeSortingBlockInputStream::remerge() sum_rows_in_blocks = new_sum_rows_in_blocks; sum_bytes_in_blocks = new_sum_bytes_in_blocks; } - - -FinishSortingBlockInputStream::FinishSortingBlockInputStream( - const BlockInputStreamPtr & input, SortDescription & description_sorted_, - SortDescription & description_to_sort_, - size_t max_merged_block_size_, size_t limit_) - : description_sorted(description_sorted_), description_to_sort(description_to_sort_), - max_merged_block_size(max_merged_block_size_), limit(limit_) -{ - children.push_back(input); - header = children.at(0)->getHeader(); - removeConstantsFromSortDescription(header, description_sorted); - removeConstantsFromSortDescription(header, description_to_sort); -} - - -struct Less -{ - const ColumnsWithSortDescriptions & left_columns; - const ColumnsWithSortDescriptions & right_columns; - - Less(const ColumnsWithSortDescriptions & left_columns_, const ColumnsWithSortDescriptions & right_columns_) : - left_columns(left_columns_), right_columns(right_columns_) {} - - bool operator() (size_t a, size_t b) const - { - for (auto it = left_columns.begin(), jt = right_columns.begin(); it != left_columns.end(); ++it, ++jt) - { - int res = it->second.direction * it->first->compareAt(a, b, *jt->first, it->second.nulls_direction); - if (res < 0) - return true; - else if (res > 0) - return false; - } - return false; - } -}; - -Block FinishSortingBlockInputStream::readImpl() -{ - if (limit && total_rows_processed == limit) - return {}; - - Block res; - if (impl) - res = impl->read(); - - /// If res block is empty, we finish sorting previous chunk of blocks. - if (!res) - { - if (end_of_stream) - return {}; - - blocks.clear(); - if (tail_block) - blocks.push_back(std::move(tail_block)); - - Block block; - size_t tail_pos = 0; - while (true) - { - block = children.back()->read(); - - /// End of input stream, but we can`t return immediatly, we need to merge already read blocks. - /// Check it later, when get end of stream from impl. - if (!block) - { - end_of_stream = true; - break; - } - - // If there were only const columns in sort description, then there is no need to sort. - // Return the blocks as is. - if (description_to_sort.empty()) - return block; - - size_t size = block.rows(); - if (size == 0) - continue; - - removeConstantsFromBlock(block); - - /// Find the position of last already read key in current block. - if (!blocks.empty()) - { - const Block & last_block = blocks.back(); - auto last_columns = getColumnsWithSortDescription(last_block, description_sorted); - auto current_columns = getColumnsWithSortDescription(block, description_sorted); - - Less less(last_columns, current_columns); - - IColumn::Permutation perm(size); - for (size_t i = 0; i < size; ++i) - perm[i] = i; - - auto it = std::upper_bound(perm.begin(), perm.end(), last_block.rows() - 1, less); - if (it != perm.end()) - { - tail_pos = it - perm.begin(); - break; - } - } - - /// If we reach here, that means that current block is first in chunk - /// or it all consists of rows with the same key as tail of a previous block. - blocks.push_back(block); - } - - /// We need to save tail of block, because next block may starts with the same key as in tail - /// and we should sort these rows in one chunk. - if (block) - { - Block head_block = block.cloneEmpty(); - tail_block = block.cloneEmpty(); - for (size_t i = 0; i < block.columns(); ++i) - { - head_block.getByPosition(i).column = block.getByPosition(i).column->cut(0, tail_pos); - tail_block.getByPosition(i).column = block.getByPosition(i).column->cut(tail_pos, block.rows() - tail_pos); - } - if (head_block.rows()) - blocks.push_back(head_block); - } - - impl = std::make_unique(blocks, description_to_sort, max_merged_block_size, limit); - res = impl->read(); - } - - if (res) - enrichBlockWithConstants(res, header); - - total_rows_processed += res.rows(); - - return res; -} - } diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.h b/dbms/src/DataStreams/MergeSortingBlockInputStream.h index 9b1a590e5f7..033a695ac27 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.h +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.h @@ -130,46 +130,4 @@ private: /// If remerge doesn't save memory at least several times, mark it as useless and don't do it anymore. bool remerge_is_useful = true; }; - - -/** Takes stream already sorted by `x` and finishes sorting it by (`x`, `y`). - * During sorting only blocks with rows that equal by `x` saved in RAM. - * */ -class FinishSortingBlockInputStream : public IProfilingBlockInputStream -{ -public: - /// limit - if not 0, allowed to return just first 'limit' rows in sorted order. - FinishSortingBlockInputStream(const BlockInputStreamPtr & input, SortDescription & description_sorted_, - SortDescription & description_to_sort_, - size_t max_merged_block_size_, size_t limit_); - - String getName() const override { return "FinishMergeSorting"; } - - bool isSortedOutput() const override { return true; } - const SortDescription & getSortDescription() const override { return description_to_sort; } - - Block getHeader() const override { return header; } - -protected: - Block readImpl() override; - -private: - SortDescription description_sorted; - SortDescription description_to_sort; - size_t max_merged_block_size; - size_t limit; - - Block tail_block; - Blocks blocks; - - std::unique_ptr impl; - - /// Before operation, will remove constant columns from blocks. And after, place constant columns back. - /// to avoid excessive virtual function calls - /// Save original block structure here. - Block header; - - bool end_of_stream = false; - size_t total_rows_processed = 0; -}; } diff --git a/dbms/src/DataStreams/processConstants.cpp b/dbms/src/DataStreams/processConstants.cpp new file mode 100644 index 00000000000..44d697b47a3 --- /dev/null +++ b/dbms/src/DataStreams/processConstants.cpp @@ -0,0 +1,50 @@ +#include + +namespace DB +{ + +void removeConstantsFromBlock(Block & block) +{ + size_t columns = block.columns(); + size_t i = 0; + while (i < columns) + { + if (block.getByPosition(i).column->isColumnConst()) + { + block.erase(i); + --columns; + } + else + ++i; + } +} + + +void removeConstantsFromSortDescription(const Block & header, SortDescription & description) +{ + description.erase(std::remove_if(description.begin(), description.end(), + [&](const SortColumnDescription & elem) + { + if (!elem.column_name.empty()) + return header.getByName(elem.column_name).column->isColumnConst(); + else + return header.safeGetByPosition(elem.column_number).column->isColumnConst(); + }), description.end()); +} + + +void enrichBlockWithConstants(Block & block, const Block & header) +{ + size_t rows = block.rows(); + size_t columns = header.columns(); + + for (size_t i = 0; i < columns; ++i) + { + const auto & col_type_name = header.getByPosition(i); + if (col_type_name.column->isColumnConst()) + block.insert(i, {col_type_name.column->cloneResized(rows), col_type_name.type, col_type_name.name}); + } +} + + +} \ No newline at end of file diff --git a/dbms/src/DataStreams/processConstants.h b/dbms/src/DataStreams/processConstants.h new file mode 100644 index 00000000000..39aff39a10d --- /dev/null +++ b/dbms/src/DataStreams/processConstants.h @@ -0,0 +1,23 @@ +#pragma once + +#include +#include + + +namespace DB +{ +/** Functions for manipulate constansts for sorting. + * See MergeSortingBlocksBlockInputStream and FinishSortingBlockInputStream for details. +*/ + +/** Remove constant columns from block. + */ +void removeConstantsFromBlock(Block & block); + +void removeConstantsFromSortDescription(const Block & header, SortDescription & description); + +/** Add into block, whose constant columns was removed by previous function, + * constant columns from header (which must have structure as before removal of constants from block). + */ +void enrichBlockWithConstants(Block & block, const Block & header); +} \ No newline at end of file diff --git a/dbms/src/DataStreams/tests/finish_sorting_stream.cpp b/dbms/src/DataStreams/tests/finish_sorting_stream.cpp index 39be70715df..d19bcb0c5b5 100644 --- a/dbms/src/DataStreams/tests/finish_sorting_stream.cpp +++ b/dbms/src/DataStreams/tests/finish_sorting_stream.cpp @@ -7,6 +7,7 @@ #include #include +#include #include @@ -96,9 +97,9 @@ int main(int argc, char ** argv) << std::endl; } } - catch (const Exception & e) { std::cerr << e.displayText() << std::endl; + return -1; } return 0; From 6ff8a604605b01da689e375672616b33a93a017c Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 19 Oct 2018 16:04:50 +0300 Subject: [PATCH 63/79] fix style --- dbms/src/Core/SortDescription.h | 1 - .../DataStreams/FinishSortingBlockInputStream.cpp | 14 ++++++-------- .../DataStreams/FinishSortingBlockInputStream.h | 2 +- dbms/src/DataStreams/processConstants.cpp | 4 +--- dbms/src/DataStreams/processConstants.h | 4 ++-- .../DataStreams/tests/finish_sorting_stream.cpp | 2 +- 6 files changed, 11 insertions(+), 16 deletions(-) diff --git a/dbms/src/Core/SortDescription.h b/dbms/src/Core/SortDescription.h index e0d3da301db..ebf3a401d9b 100644 --- a/dbms/src/Core/SortDescription.h +++ b/dbms/src/Core/SortDescription.h @@ -43,4 +43,3 @@ struct SortColumnDescription using SortDescription = std::vector; } - diff --git a/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp b/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp index 67ea04fc2e5..704b18ab98c 100644 --- a/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp +++ b/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp @@ -29,10 +29,10 @@ FinishSortingBlockInputStream::FinishSortingBlockInputStream( size_t max_merged_block_size_, size_t limit_) : description_sorted(description_sorted_), description_to_sort(description_to_sort_), max_merged_block_size(max_merged_block_size_), limit(limit_) - { if (!isPrefix(description_sorted, description_to_sort)) - throw Exception("Can`t finish sorting. SortDescription of already sorted streamis not prefix of SortDescription needed to sort", ErrorCodes::LOGICAL_ERROR); + throw Exception("Can`t finish sorting. SortDescription of already sorted stream is not prefix of" + "SortDescription needed to sort", ErrorCodes::LOGICAL_ERROR); children.push_back(input); header = children.at(0)->getHeader(); @@ -81,11 +81,9 @@ Block FinishSortingBlockInputStream::readImpl() if (tail_block) blocks.push_back(std::move(tail_block)); - Block block; - size_t tail_pos = 0; while (true) { - block = children.back()->read(); + Block block = children.back()->read(); /// End of input stream, but we can`t return immediatly, we need to merge already read blocks. /// Check it later, when get end of stream from impl. @@ -104,7 +102,7 @@ Block FinishSortingBlockInputStream::readImpl() if (size == 0) continue; - /// We need to sort each block separatly before merging. + /// We need to sort each block separatly before merging. sortBlock(block, description_to_sort); removeConstantsFromBlock(block); @@ -128,7 +126,7 @@ Block FinishSortingBlockInputStream::readImpl() /// and we should sort these rows in one chunk. if (it != perm.end()) { - tail_pos = it - perm.begin(); + size_t tail_pos = it - perm.begin(); Block head_block = block.cloneEmpty(); tail_block = block.cloneEmpty(); @@ -161,4 +159,4 @@ Block FinishSortingBlockInputStream::readImpl() return res; } -} \ No newline at end of file +} diff --git a/dbms/src/DataStreams/FinishSortingBlockInputStream.h b/dbms/src/DataStreams/FinishSortingBlockInputStream.h index 5afaa5f1512..6688eefbca7 100644 --- a/dbms/src/DataStreams/FinishSortingBlockInputStream.h +++ b/dbms/src/DataStreams/FinishSortingBlockInputStream.h @@ -48,4 +48,4 @@ private: bool end_of_stream = false; size_t total_rows_processed = 0; }; -} \ No newline at end of file +} diff --git a/dbms/src/DataStreams/processConstants.cpp b/dbms/src/DataStreams/processConstants.cpp index 44d697b47a3..fff10afe780 100644 --- a/dbms/src/DataStreams/processConstants.cpp +++ b/dbms/src/DataStreams/processConstants.cpp @@ -45,6 +45,4 @@ void enrichBlockWithConstants(Block & block, const Block & header) block.insert(i, {col_type_name.column->cloneResized(rows), col_type_name.type, col_type_name.name}); } } - - -} \ No newline at end of file +} diff --git a/dbms/src/DataStreams/processConstants.h b/dbms/src/DataStreams/processConstants.h index 39aff39a10d..6fedab75c76 100644 --- a/dbms/src/DataStreams/processConstants.h +++ b/dbms/src/DataStreams/processConstants.h @@ -6,7 +6,7 @@ namespace DB { -/** Functions for manipulate constansts for sorting. +/** Functions for manipulate constants for sorting. * See MergeSortingBlocksBlockInputStream and FinishSortingBlockInputStream for details. */ @@ -20,4 +20,4 @@ void removeConstantsFromSortDescription(const Block & header, SortDescription & * constant columns from header (which must have structure as before removal of constants from block). */ void enrichBlockWithConstants(Block & block, const Block & header); -} \ No newline at end of file +} diff --git a/dbms/src/DataStreams/tests/finish_sorting_stream.cpp b/dbms/src/DataStreams/tests/finish_sorting_stream.cpp index d19bcb0c5b5..7627d1712b2 100644 --- a/dbms/src/DataStreams/tests/finish_sorting_stream.cpp +++ b/dbms/src/DataStreams/tests/finish_sorting_stream.cpp @@ -57,7 +57,6 @@ int main(int argc, char ** argv) sort_descr_final.emplace_back("col1", 1, 1); sort_descr_final.emplace_back("col2", 1, 1); - stream = std::make_shared(stream, sort_descr_final); stream = std::make_shared(stream, sort_descr, sort_descr_final, n, 0); { @@ -97,6 +96,7 @@ int main(int argc, char ** argv) << std::endl; } } + catch (const Exception & e) { std::cerr << e.displayText() << std::endl; return -1; From 4449d7e05942e9455b46cb7e33f89afde5039965 Mon Sep 17 00:00:00 2001 From: Alex Zatelepin Date: Fri, 19 Oct 2018 16:21:50 +0300 Subject: [PATCH 64/79] Update FinishSortingBlockInputStream.cpp --- dbms/src/DataStreams/FinishSortingBlockInputStream.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp b/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp index 704b18ab98c..7a16737f928 100644 --- a/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp +++ b/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp @@ -31,7 +31,7 @@ FinishSortingBlockInputStream::FinishSortingBlockInputStream( max_merged_block_size(max_merged_block_size_), limit(limit_) { if (!isPrefix(description_sorted, description_to_sort)) - throw Exception("Can`t finish sorting. SortDescription of already sorted stream is not prefix of" + throw Exception("Can`t finish sorting. SortDescription of already sorted stream is not prefix of " "SortDescription needed to sort", ErrorCodes::LOGICAL_ERROR); children.push_back(input); From dcf04ac36dd976bd99ec487343191f5c093264e8 Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 19 Oct 2018 18:33:40 +0300 Subject: [PATCH 65/79] unit-test for ExpressionAnalyzer [CLICKHOUSE-3996] --- dbms/src/Interpreters/ExpressionAnalyzer.h | 3 + dbms/src/Interpreters/tests/CMakeLists.txt | 4 + .../tests/expression_analyzer.cpp | 148 ++++++++++++++++++ 3 files changed, 155 insertions(+) create mode 100644 dbms/src/Interpreters/tests/expression_analyzer.cpp diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 732295fe449..979ab330c2e 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -209,6 +209,9 @@ public: */ const Tables & getExternalTables() const { return external_tables; } + /// Get intermediates for tests + const ExpressionAnalyzerData & getAnalyzedData() const { return *this; } + /// Create Set-s that we can from IN section to use the index on them. void makeSetsForIndex(); diff --git a/dbms/src/Interpreters/tests/CMakeLists.txt b/dbms/src/Interpreters/tests/CMakeLists.txt index fb79250cd7c..01fc72730dc 100644 --- a/dbms/src/Interpreters/tests/CMakeLists.txt +++ b/dbms/src/Interpreters/tests/CMakeLists.txt @@ -45,6 +45,10 @@ add_executable (in_join_subqueries_preprocessor in_join_subqueries_preprocessor. target_link_libraries (in_join_subqueries_preprocessor dbms) add_check(in_join_subqueries_preprocessor) +add_executable (expression_analyzer expression_analyzer.cpp) +target_link_libraries (expression_analyzer dbms clickhouse_storages_system) +add_check(expression_analyzer) + add_executable (users users.cpp) target_link_libraries (users dbms ${Boost_FILESYSTEM_LIBRARY}) diff --git a/dbms/src/Interpreters/tests/expression_analyzer.cpp b/dbms/src/Interpreters/tests/expression_analyzer.cpp new file mode 100644 index 00000000000..a523e3f43b8 --- /dev/null +++ b/dbms/src/Interpreters/tests/expression_analyzer.cpp @@ -0,0 +1,148 @@ +#include + +#include +#include +#include + +#include +#include + +#include +#include + +#include +#include +#include + +#include +#include + +#include +#include +#include + + +using namespace DB; + +struct TestEntry +{ + String query; + std::unordered_map expected_aliases; /// alias -> AST.getID() + NamesAndTypesList source_columns = {}; + Names required_result_columns = {}; + + bool check(const Context & context) + { + ASTPtr ast = parse(query); + + ExpressionAnalyzer analyzer(ast, context, {}, source_columns, required_result_columns); + + const ExpressionAnalyzerData & data = analyzer.getAnalyzedData(); + + if (!checkAliases(data)) + { + collectWithAnalysers(context, ast); + return false; + } + + return true; + } + +private: + bool checkAliases(const ExpressionAnalyzerData & data) + { + for (const auto & alias : data.aliases) + { + const String & alias_name = alias.first; + if (expected_aliases.count(alias_name) == 0 || + expected_aliases[alias_name] != alias.second->getID()) + { + std::cout << "unexpected alias: " << alias_name << ' ' << alias.second->getID() << std::endl; + return false; + } + else + expected_aliases.erase(alias_name); + } + + if (!expected_aliases.empty()) + { + std::cout << "missing aliases: " << expected_aliases.size() << std::endl; + return false; + } + + return true; + } + + static ASTPtr parse(const std::string & query) + { + ParserSelectQuery parser; + std::string message; + auto text = query.data(); + if (ASTPtr ast = tryParseQuery(parser, text, text + query.size(), message, false, "", false, 0)) + return ast; + throw Exception(message); + } + + void collectWithAnalysers(const Context & context, ASTPtr ast) const + { + ReadBufferFromFileDescriptor in(STDIN_FILENO); + WriteBufferFromFileDescriptor out(STDOUT_FILENO); + + CollectAliases collect_aliases; + collect_aliases.process(ast); + + ExecuteTableFunctions execute_table_functions; + execute_table_functions.process(ast, context); + + CollectTables collect_tables; + collect_tables.process(ast, context, collect_aliases, execute_table_functions); + collect_tables.dump(out); + } +}; + + +int main() +{ + std::vector queries = + { + { + "SELECT number AS n FROM system.numbers LIMIT 0", + {{"n", "Identifier_number"}}, + { NameAndTypePair("number", std::make_shared()) } + }, + + { + "SELECT number AS n FROM system.numbers LIMIT 0", + {{"n", "Identifier_number"}} + } + }; + + Context context = Context::createGlobal(); + + auto system_database = std::make_shared("system"); + context.addDatabase("system", system_database); + //context.setCurrentDatabase("system"); + system_database->attachTable("one", StorageSystemOne::create("one")); + system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false)); + + size_t success = 0; + for (auto & entry : queries) + { + try + { + if (entry.check(context)) + { + ++success; + std::cout << "[OK] " << entry.query << std::endl; + } + else + std::cout << "[Failed] " << entry.query << std::endl; + } + catch (Exception & e) + { + std::cout << "[Error] " << entry.query << std::endl << e.displayText() << std::endl; + } + } + + return success != queries.size(); +} From b784f754a703277641048f98800563371a4b5873 Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 19 Oct 2018 18:42:47 +0300 Subject: [PATCH 66/79] minor settings changes --- dbms/src/Interpreters/ExpressionAnalyzer.h | 29 ++++++++++--------- .../LogicalExpressionsOptimizer.cpp | 2 +- .../LogicalExpressionsOptimizer.h | 22 +++++++------- .../PredicateExpressionsOptimizer.h | 11 +++---- 4 files changed, 33 insertions(+), 31 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 979ab330c2e..4117d8d3fe1 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -96,6 +96,15 @@ private: /// Extracts settings to enlight which are used (and avoid copy of others). struct ExtractedSettings { + /// for QueryNormalizer + const UInt64 max_ast_depth; + const UInt64 max_expanded_ast_elements; + const String count_distinct_implementation; + + /// for PredicateExpressionsOptimizer + const bool enable_optimize_predicate_expression; + + /// for ExpressionAnalyzer const bool asterisk_left_columns_only; const bool use_index_for_in_with_subqueries; const bool enable_conditional_computation; @@ -105,27 +114,19 @@ private: const String join_default_strictness; const UInt64 min_equality_disjunction_chain_length; - /// for PredicateExpressionsOptimizer - const bool enable_optimize_predicate_expression; - - /// for QueryNormalizer - const UInt64 max_ast_depth; - const UInt64 max_expanded_ast_elements; - const String count_distinct_implementation; - ExtractedSettings(const Settings & settings) - : asterisk_left_columns_only(settings.asterisk_left_columns_only), + : max_ast_depth(settings.max_ast_depth), + max_expanded_ast_elements(settings.max_expanded_ast_elements), + count_distinct_implementation(settings.count_distinct_implementation), + enable_optimize_predicate_expression(settings.enable_optimize_predicate_expression), + asterisk_left_columns_only(settings.asterisk_left_columns_only), use_index_for_in_with_subqueries(settings.use_index_for_in_with_subqueries), enable_conditional_computation(settings.enable_conditional_computation), join_use_nulls(settings.join_use_nulls), size_limits_for_set(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode), size_limits_for_join(settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode), join_default_strictness(settings.join_default_strictness.toString()), - min_equality_disjunction_chain_length(settings.optimize_min_equality_disjunction_chain_length), - enable_optimize_predicate_expression(settings.enable_optimize_predicate_expression), - max_ast_depth(settings.max_ast_depth), - max_expanded_ast_elements(settings.max_expanded_ast_elements), - count_distinct_implementation(settings.count_distinct_implementation) + min_equality_disjunction_chain_length(settings.optimize_min_equality_disjunction_chain_length) {} }; diff --git a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp index 9705b19862b..42176788cbd 100644 --- a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp @@ -30,7 +30,7 @@ bool LogicalExpressionsOptimizer::OrWithExpression::operator<(const OrWithExpres return std::tie(this->or_function, this->expression) < std::tie(rhs.or_function, rhs.expression); } -LogicalExpressionsOptimizer::LogicalExpressionsOptimizer(ASTSelectQuery * select_query_, LogicalExpressionsOptimizerSettings && settings_) +LogicalExpressionsOptimizer::LogicalExpressionsOptimizer(ASTSelectQuery * select_query_, ExtractedSettings && settings_) : select_query(select_query_), settings(settings_) { } diff --git a/dbms/src/Interpreters/LogicalExpressionsOptimizer.h b/dbms/src/Interpreters/LogicalExpressionsOptimizer.h index a559ec5588a..636c83e1d9f 100644 --- a/dbms/src/Interpreters/LogicalExpressionsOptimizer.h +++ b/dbms/src/Interpreters/LogicalExpressionsOptimizer.h @@ -16,15 +16,6 @@ struct Settings; class ASTFunction; class ASTSelectQuery; -struct LogicalExpressionsOptimizerSettings -{ - const UInt64 optimize_min_equality_disjunction_chain_length; - - LogicalExpressionsOptimizerSettings(UInt64 optimize_min_equality_disjunction_chain_length_) - : optimize_min_equality_disjunction_chain_length(optimize_min_equality_disjunction_chain_length_) - {} -}; - /** This class provides functions for optimizing boolean expressions within queries. * @@ -34,9 +25,18 @@ struct LogicalExpressionsOptimizerSettings */ class LogicalExpressionsOptimizer final { + struct ExtractedSettings + { + const UInt64 optimize_min_equality_disjunction_chain_length; + + ExtractedSettings(UInt64 optimize_min_equality_disjunction_chain_length_) + : optimize_min_equality_disjunction_chain_length(optimize_min_equality_disjunction_chain_length_) + {} + }; + public: /// Constructor. Accepts the root of the query DAG. - LogicalExpressionsOptimizer(ASTSelectQuery * select_query_, LogicalExpressionsOptimizerSettings && settings_); + LogicalExpressionsOptimizer(ASTSelectQuery * select_query_, ExtractedSettings && settings_); /** Replace all rather long homogeneous OR-chains expr = x1 OR ... OR expr = xN * on the expressions `expr` IN (x1, ..., xN). @@ -100,7 +100,7 @@ private: private: ASTSelectQuery * select_query; - const LogicalExpressionsOptimizerSettings settings; + const ExtractedSettings settings; /// Information about the OR-chains inside the query. DisjunctiveEqualityChainsMap disjunctive_equality_chains_map; /// Number of processed OR-chains. diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h index 298a4b581c8..c1a02a7df18 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h @@ -44,19 +44,20 @@ class PredicateExpressionsOptimizer /// Extracts settings, mostly to show which are used and which are not. struct ExtractedSettings { - const bool enable_optimize_predicate_expression; - /// QueryNormalizer settings const UInt64 max_ast_depth; const UInt64 max_expanded_ast_elements; const String count_distinct_implementation; + /// for PredicateExpressionsOptimizer + const bool enable_optimize_predicate_expression; + template ExtractedSettings(const T & settings) - : enable_optimize_predicate_expression(settings.enable_optimize_predicate_expression), - max_ast_depth(settings.max_ast_depth), + : max_ast_depth(settings.max_ast_depth), max_expanded_ast_elements(settings.max_expanded_ast_elements), - count_distinct_implementation(settings.count_distinct_implementation) + count_distinct_implementation(settings.count_distinct_implementation), + enable_optimize_predicate_expression(settings.enable_optimize_predicate_expression) {} }; From 31794b7a46bfe7ed45113230911ff10802720ddc Mon Sep 17 00:00:00 2001 From: VadimPE Date: Mon, 22 Oct 2018 14:58:56 +0300 Subject: [PATCH 67/79] fix codestyle --- dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 69b088e4aa9..369b6bf9f1e 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1400,7 +1400,8 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( queue_.pullLogsToQueue(zookeeper); Coordination::GetResponse quorum_last_part_response = quorum_last_part_future.get(); - if (!quorum_last_part_response.error) { + if (!quorum_last_part_response.error) + { ReplicatedMergeTreeQuorumAddedParts parts_with_quorum(queue.format_version); if (!quorum_last_part_response.data.empty()) { From 91913561586993275dc30132295545983381d633 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Oct 2018 17:31:25 +0300 Subject: [PATCH 68/79] CLICKHOUSE-4080: Add simple stress-test script and docker image for it --- docker/test/stress/Dockerfile | 28 +++++++++++++ docker/test/stress/README.md | 31 +++++++++++++++ docker/test/stress/stress | 75 +++++++++++++++++++++++++++++++++++ 3 files changed, 134 insertions(+) create mode 100644 docker/test/stress/Dockerfile create mode 100644 docker/test/stress/README.md create mode 100755 docker/test/stress/stress diff --git a/docker/test/stress/Dockerfile b/docker/test/stress/Dockerfile new file mode 100644 index 00000000000..a32131b707e --- /dev/null +++ b/docker/test/stress/Dockerfile @@ -0,0 +1,28 @@ +FROM ubuntu:18.04 + +RUN apt-get update -y \ + && env DEBIAN_FRONTEND=noninteractive \ + apt-get install --yes --no-install-recommends \ + bash \ + tzdata \ + fakeroot \ + debhelper \ + parallel \ + expect \ + python \ + python-lxml \ + python-termcolor \ + python-requests \ + curl \ + sudo \ + openssl \ + netcat-openbsd \ + telnet + +COPY ./stress /stress + +CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ + dpkg -i package_folder/clickhouse-server_*.deb; \ + dpkg -i package_folder/clickhouse-client_*.deb; \ + dpkg -i package_folder/clickhouse-test_*.deb; \ + service clickhouse-server start && ./stress --output-folder test_output \ No newline at end of file diff --git a/docker/test/stress/README.md b/docker/test/stress/README.md new file mode 100644 index 00000000000..c9b6da37b05 --- /dev/null +++ b/docker/test/stress/README.md @@ -0,0 +1,31 @@ +Allow to run simple ClickHouse stress test in Docker from debian packages. +Actually it runs single copy of clickhouse-performance-test and multiple copies +of clickhouse-test (functional tests). This allows to find problems like +segmentation fault which cause shutdown of server. + +Usage: +``` +$ ls $HOME/someclickhouse +clickhouse-client_18.14.9_all.deb clickhouse-common-static_18.14.9_amd64.deb clickhouse-server_18.14.9_all.deb clickhouse-test_18.14.9_all.deb +$ docker run --volume=$HOME/someclickhouse:/package_folder --volume=$HOME/test_output:/test_output yandex/clickhouse-stress-test +Selecting previously unselected package clickhouse-common-static. +(Reading database ... 14442 files and directories currently installed.) +... +Start clickhouse-server service: Path to data directory in /etc/clickhouse-server/config.xml: /var/lib/clickhouse/ +DONE +2018-10-22 13:40:35,744 Will wait functests to finish +2018-10-22 13:40:40,747 Finished 0 from 16 processes +2018-10-22 13:40:45,751 Finished 0 from 16 processes +... +2018-10-22 13:49:11,165 Finished 15 from 16 processes +2018-10-22 13:49:16,171 Checking ClickHouse still alive +Still alive +2018-10-22 13:49:16,195 Stress is ok +2018-10-22 13:49:16,195 Copying server log files +$ ls $HOME/test_result +clickhouse-server.err.log clickhouse-server.log.0.gz stderr.log stress_test_run_0.txt stress_test_run_11.txt stress_test_run_13.txt +stress_test_run_15.txt stress_test_run_2.txt stress_test_run_4.txt stress_test_run_6.txt stress_test_run_8.txt clickhouse-server.log +perf_stress_run.txt stdout.log stress_test_run_10.txt stress_test_run_12.txt +stress_test_run_14.txt stress_test_run_1.txt +stress_test_run_3.txt stress_test_run_5.txt stress_test_run_7.txt stress_test_run_9.txt +``` diff --git a/docker/test/stress/stress b/docker/test/stress/stress new file mode 100755 index 00000000000..9a8c236234c --- /dev/null +++ b/docker/test/stress/stress @@ -0,0 +1,75 @@ +#!/usr/bin/env python +#-*- coding: utf-8 -*- +from multiprocessing import cpu_count +from subprocess import Popen, check_call +import os +import shutil +import argparse +import logging +import time + +def run_perf_test(cmd, xmls_path, output_folder): + output_path = os.path.join(output_folder, "perf_stress_run.txt") + f = open(output_path, 'w') + p = Popen("{} --skip-tags=long --r {}".format(cmd, xmls_path), shell=True, stdout=f, stderr=f) + return p + +def run_func_test(cmd, output_prefix, num_processes): + output_paths = [os.path.join(output_prefix, "stress_test_run_{}.txt".format(i)) for i in range(num_processes)] + f = open(output_paths[0], 'w') + pipes = [Popen("{}".format(cmd), shell=True, stdout=f, stderr=f)] + for output_path in output_paths[1:]: + time.sleep(0.5) + f = open(output_path, 'w') + p = Popen("{} --order=random".format(cmd), shell=True, stdout=f, stderr=f) + pipes.append(p) + return pipes + +def check_clickhouse_alive(cmd): + try: + logging.info("Checking ClickHouse still alive") + check_call("{} --query \"select 'Still alive'\"".format(cmd), shell=True) + return True + except: + return False + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO, format='%(asctime)s %(message)s') + parser = argparse.ArgumentParser(description="ClickHouse script for running stresstest") + parser.add_argument("--test-cmd", default='clickhouse-test') + parser.add_argument("--client-cmd", default='clickhouse-client') + parser.add_argument("--perf-test-cmd", default='clickhouse-performance-test') + parser.add_argument("--perf-test-xml-path", default='/usr/share/clickhouse-test/performance/') + parser.add_argument("--server-log-folder", default='/var/log/clickhouse-server') + parser.add_argument("--output-folder") + parser.add_argument("--num-parallel", default=cpu_count() // 3); + + args = parser.parse_args() + func_pipes = [] + perf_process = None + try: + perf_process = run_perf_test(args.perf_test_cmd, args.perf_test_xml_path, args.output_folder) + func_pipes = run_func_test(args.test_cmd, args.output_folder, args.num_parallel) + + logging.info("Will wait functests to finish") + while True: + retcodes = [] + for p in func_pipes: + if p.poll(): + retcodes.append(p.returncode) + if len(retcodes) == len(func_pipes): + break + logging.info("Finished %s from %s processes", len(retcodes), len(func_pipes)) + time.sleep(5) + + if not check_clickhouse_alive(args.client_cmd): + raise Exception("Stress failed, results in logs") + else: + logging.info("Stress is ok") + except Exception as ex: + raise ex + finally: + if os.path.exists(args.server_log_folder): + logging.info("Copying server log files") + for log_file in os.listdir(args.server_log_folder): + shutil.copy(os.path.join(args.server_log_folder, log_file), os.path.join(args.output_folder, log_file)) From 68352d8d63fa0d149059ada74d56f96b61ea7596 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Oct 2018 18:23:03 +0300 Subject: [PATCH 69/79] Add small sleep before test run --- docker/test/stress/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/Dockerfile b/docker/test/stress/Dockerfile index a32131b707e..8954c41efae 100644 --- a/docker/test/stress/Dockerfile +++ b/docker/test/stress/Dockerfile @@ -25,4 +25,4 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ dpkg -i package_folder/clickhouse-server_*.deb; \ dpkg -i package_folder/clickhouse-client_*.deb; \ dpkg -i package_folder/clickhouse-test_*.deb; \ - service clickhouse-server start && ./stress --output-folder test_output \ No newline at end of file + service clickhouse-server start && sleep 1 && ./stress --output-folder test_output \ No newline at end of file From dc8b3a779199f4efe723a7b6a94f39213430bbcc Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 22 Oct 2018 23:33:32 +0800 Subject: [PATCH 70/79] Fix build without jemalloc ``` libs/libglibc-compatibility/libglibc-compatibility.a(glibc-compatibility.c.o): In function `explicit_bzero': /home/amos/git/chorigin/build-dev/../libs/libglibc-compatibility/glibc-compatibility.c:151: multiple definition of `explicit_bzero' contrib/ssl/crypto/libcrypto.a(explicit_bzero.c.o):/home/amos/git/chorigin/build-dev/../contrib/ssl/crypto/compat/explicit_bzero.c:16: first defined here collect2: error: ld returned 1 exit status ``` --- libs/libglibc-compatibility/glibc-compatibility.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/libglibc-compatibility/glibc-compatibility.c b/libs/libglibc-compatibility/glibc-compatibility.c index ad88d3254bf..74bf4346a8c 100644 --- a/libs/libglibc-compatibility/glibc-compatibility.c +++ b/libs/libglibc-compatibility/glibc-compatibility.c @@ -147,7 +147,7 @@ const char * __shm_directory(size_t * len) * OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN * CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. */ -void explicit_bzero(void * buf, size_t len) +void __attribute__((__weak__)) explicit_bzero(void * buf, size_t len) { memset(buf, 0, len); __asm__ __volatile__("" :: "r"(buf) : "memory"); From b180c381c401b02244f106dbb7f5f73b835881f9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Oct 2018 19:05:52 +0300 Subject: [PATCH 71/79] Change default threads value --- docker/test/stress/stress | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 9a8c236234c..bb395a786ba 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -42,7 +42,7 @@ if __name__ == "__main__": parser.add_argument("--perf-test-xml-path", default='/usr/share/clickhouse-test/performance/') parser.add_argument("--server-log-folder", default='/var/log/clickhouse-server') parser.add_argument("--output-folder") - parser.add_argument("--num-parallel", default=cpu_count() // 3); + parser.add_argument("--num-parallel", default=cpu_count() // 2); args = parser.parse_args() func_pipes = [] From e20f38a855c3f370bb97eba862e495fe599965d8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Oct 2018 20:04:43 +0300 Subject: [PATCH 72/79] Change default threads, num_procs/3 is enought --- docker/test/stress/stress | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index bb395a786ba..9a8c236234c 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -42,7 +42,7 @@ if __name__ == "__main__": parser.add_argument("--perf-test-xml-path", default='/usr/share/clickhouse-test/performance/') parser.add_argument("--server-log-folder", default='/var/log/clickhouse-server') parser.add_argument("--output-folder") - parser.add_argument("--num-parallel", default=cpu_count() // 2); + parser.add_argument("--num-parallel", default=cpu_count() // 3); args = parser.parse_args() func_pipes = [] From e3c2a2afd9546ab56edf881e02b4b014f23b95dd Mon Sep 17 00:00:00 2001 From: BayoNet Date: Mon, 22 Oct 2018 21:45:23 +0300 Subject: [PATCH 73/79] Clarifications for MySQL table engine and toStartOfMonth function (#3440) * Update of english version of descriprion of the table function `file`. * New syntax for ReplacingMergeTree. Some improvements in text. * Significantly change article about SummingMergeTree. Article is restructured, text is changed in many places of the document. New syntax for table creation is described. * Descriptions of AggregateFunction and AggregatingMergeTree are updated. Russian version. * New syntax for new syntax of CREATE TABLE * Added english docs on Aggregating, Replacing and SummingMergeTree. * CollapsingMergeTree docs. English version. * 1. Update of CollapsingMergeTree. 2. Minor changes in markup * Update aggregatefunction.md * Update aggregatefunction.md * Update aggregatefunction.md * Update aggregatingmergetree.md * GraphiteMergeTree docs update. New syntax for creation of Replicated* tables. Minor changes in *MergeTree tables creation syntax. * Markup fix * Markup and language fixes * Clarification in the CollapsingMergeTree article * Clarifications for MySQL table engine and toStartOfMonth function --- docs/en/operations/table_engines/mysql.md | 8 ++++++-- .../query_language/dicts/external_dicts_dict_sources.md | 1 + docs/en/query_language/functions/date_time_functions.md | 3 +++ docs/ru/operations/table_engines/mysql.md | 6 +++++- docs/ru/query_language/functions/date_time_functions.md | 3 +++ 5 files changed, 18 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/table_engines/mysql.md b/docs/en/operations/table_engines/mysql.md index b9e419b5ac0..9c3a84a4b6e 100644 --- a/docs/en/operations/table_engines/mysql.md +++ b/docs/en/operations/table_engines/mysql.md @@ -18,9 +18,13 @@ MySQL('host:port', 'database', 'table', 'user', 'password'[, replace_query, 'on_ - `user` — The MySQL User. - `password` — User password. - `replace_query` — Flag that sets query substitution `INSERT INTO` to `REPLACE INTO`. If `replace_query=1`, the query is replaced. -- `'on_duplicate_clause'` — Adds the `ON DUPLICATE KEY UPDATE 'on_duplicate_clause'` expression to the `INSERT` query. For example: `impression = VALUES(impression) + impression`. To specify `'on_duplicate_clause'` you need to pass `0` to the `replace_query` parameter. If you simultaneously pass `replace_query = 1` and `'on_duplicate_clause'`, ClickHouse generates an exception. +- `on_duplicate_clause` — Adds the `ON DUPLICATE KEY on_duplicate_clause` expression to the `INSERT` query. -At this time, simple `WHERE` clauses such as ` =, !=, >, >=, <, <=` are executed on the MySQL server. + Example: `INSERT INTO t (c1,c2) VALUES ('a', 2) ON DUPLICATE KEY UPDATE c2 = c2 + 1`, where `on_duplicate_clause` is `UPDATE c2 = c2 + 1`. See MySQL documentation to find which `on_duplicate_clause` you can use with `ON DUPLICATE KEY` clause. + + To specify `on_duplicate_clause` you need to pass `0` to the `replace_query` parameter. If you simultaneously pass `replace_query = 1` and `on_duplicate_clause`, ClickHouse generates an exception. + +At this time, simple `WHERE` clauses such as ` =, !=, >, >=, <, <=` are executed on the MySQL server. The rest of the conditions and the `LIMIT` sampling constraint are executed in ClickHouse only after the query to MySQL finishes. diff --git a/docs/en/query_language/dicts/external_dicts_dict_sources.md b/docs/en/query_language/dicts/external_dicts_dict_sources.md index 977566afcc7..023f14f5a70 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_sources.md +++ b/docs/en/query_language/dicts/external_dicts_dict_sources.md @@ -400,6 +400,7 @@ Setting fields: - `db` – Name of the database. - `table` – Name of the table. - `where ` – The selection criteria. May be omitted. +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime). diff --git a/docs/en/query_language/functions/date_time_functions.md b/docs/en/query_language/functions/date_time_functions.md index 82821474431..d451cf2f9d4 100644 --- a/docs/en/query_language/functions/date_time_functions.md +++ b/docs/en/query_language/functions/date_time_functions.md @@ -60,6 +60,9 @@ Returns the date. Rounds down a date or date with time to the first day of the month. Returns the date. +!!! attention + The behavior of parsing incorrect dates is implementation specific. ClickHouse may return zero date, throw an exception or do "natural" overflow. + ## toStartOfQuarter Rounds down a date or date with time to the first day of the quarter. diff --git a/docs/ru/operations/table_engines/mysql.md b/docs/ru/operations/table_engines/mysql.md index d4444dfe269..f452d34c4a6 100644 --- a/docs/ru/operations/table_engines/mysql.md +++ b/docs/ru/operations/table_engines/mysql.md @@ -18,7 +18,11 @@ MySQL('host:port', 'database', 'table', 'user', 'password'[, replace_query, 'on_ - `user` — Пользователь MySQL. - `password` — Пароль пользователя. - `replace_query` — Флаг, устанавливающий замену запроса `INSERT INTO` на `REPLACE INTO`. Если `replace_query=1`, то запрос заменяется. -- `'on_duplicate_clause'` — Добавляет выражение `ON DUPLICATE KEY UPDATE 'on_duplicate_clause'` в запрос `INSERT`. Например, `impression = VALUES(impression) + impression`. Чтобы указать `'on_duplicate_clause'` необходимо передать `0` в параметр `replace_query`. Если одновременно передать `replace_query = 1` и `'on_duplicate_clause'`, то ClickHouse сгенерирует исключение. +- `on_duplicate_clause` — Добавляет выражение `ON DUPLICATE KEY 'on_duplicate_clause'` в запрос `INSERT`. + + Например, `INSERT INTO t (c1,c2) VALUES ('a', 2) ON DUPLICATE KEY UPDATE c2 = c2 + 1`, где `on_duplicate_clause` — `UPDATE c2 = c2 + 1`. Какие выражения `on_duplicate_clause` вы можете использовать с `ON DUPLICATE KEY`, смотрите в документации MySQL. + + Чтобы указать `on_duplicate_clause` необходимо передать `0` в параметр `replace_query`. Если одновременно передать `replace_query = 1` и `on_duplicate_clause`, то ClickHouse сгенерирует исключение. На данный момент простые условия `WHERE`, такие как `=, !=, >, >=, <, <=` будут выполняться на стороне сервера MySQL. diff --git a/docs/ru/query_language/functions/date_time_functions.md b/docs/ru/query_language/functions/date_time_functions.md index c0a654a0507..048b8a9ce8c 100644 --- a/docs/ru/query_language/functions/date_time_functions.md +++ b/docs/ru/query_language/functions/date_time_functions.md @@ -51,6 +51,9 @@ SELECT Округляет дату или дату-с-временем вниз до первого дня месяца. Возвращается дата. +!!! attention + Возвращаемое значение для некорректных дат зависит от реализации. ClickHouse может вернуть нулевую дату, выбросить исключение, или выполнить "естественное" перетекание дат между месяцами. + ## toStartOfQuarter Округляет дату или дату-с-временем вниз до первого дня квартала. Первый день квартала - это одно из 1 января, 1 апреля, 1 июля, 1 октября. From 23ea8adf9c2406f57c4c62a7315f5adf9176dbeb Mon Sep 17 00:00:00 2001 From: BayoNet Date: Mon, 22 Oct 2018 22:32:52 +0300 Subject: [PATCH 74/79] English documentation on `hasAny()` and `hasAll()` functions for arrays (#3426) * Update of english version of descriprion of the table function `file`. * New syntax for ReplacingMergeTree. Some improvements in text. * Significantly change article about SummingMergeTree. Article is restructured, text is changed in many places of the document. New syntax for table creation is described. * Descriptions of AggregateFunction and AggregatingMergeTree are updated. Russian version. * New syntax for new syntax of CREATE TABLE * Added english docs on Aggregating, Replacing and SummingMergeTree. * CollapsingMergeTree docs. English version. * 1. Update of CollapsingMergeTree. 2. Minor changes in markup * Update aggregatefunction.md * Update aggregatefunction.md * Update aggregatefunction.md * Update aggregatingmergetree.md * GraphiteMergeTree docs update. New syntax for creation of Replicated* tables. Minor changes in *MergeTree tables creation syntax. * Markup fix * Markup and language fixes * Clarification in the CollapsingMergeTree article * English documentation on hasAny and hasAll functions for arrays. * Update array_functions.md --- .../functions/array_functions.md | 87 +++++++++++++++++-- 1 file changed, 80 insertions(+), 7 deletions(-) diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index afb4f5f1e6e..cd6528743d7 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -53,7 +53,7 @@ Combines arrays passed as arguments. arrayConcat(arrays) ``` -**Arguments** +**Parameters** - `arrays` – Arrays of comma-separated `[values]`. @@ -92,6 +92,79 @@ SELECT has([1, 2, NULL], NULL) └─────────────────────────┘ ``` +## hasAll + +Checks whether one array is a subset of another. + +``` +hasAll(set, subset) +``` + +**Parameters** + +- `set` – Array of any type with a set of elements. +- `subset` – Array of any type with elements that should be tested to be a subset of `set`. + +**Return values** + +- `1`, if `set` contains all of the elements from `subset`. +- `0`, otherwise. + +**Peculiar properties** + +- An empty array is a subset of any array. +- `Null` processed as a value. +- Order of values in both of arrays doesn't matter. + +**Examples** + +`SELECT hasAll([], [])` returns 1. + +`SELECT hasAll([1, Null], [Null])` returns 1. + +`SELECT hasAll([1.0, 2, 3, 4], [1, 3])` returns 1. + +`SELECT hasAll(['a', 'b'], ['a'])` returns 1. + +`SELECT hasAll([1], ['a'])` returns 0. + +`SELECT hasAll([[1, 2], [3, 4]], [[1, 2], [3, 5]])` returns 0. + +## hasAny + +Checks whether two arrays have intersection by some elements. + +``` +hasAny(array1, array2) +``` + +**Parameters** + +- `array1` – Array of any type with a set of elements. +- `array2` – Array of any type with a set of elements. + +**Return values** + +- `1`, if `array1` and `array2` have one similar element at least. +- `0`, otherwise. + +**Peculiar properties** + +- `Null` processed as a value. +- Order of values in both of arrays doesn't matter. + +**Examples** + +`SELECT hasAny([1], [])` returns `0`. + +`SELECT hasAny([Null], [Null, 1])` returns `1`. + +`SELECT hasAny([-128, 1., 512], [1])` returns `1`. + +`SELECT hasAny([[1, 2], [3, 4]], ['a', 'c'])` returns `0`. + +`SELECT hasAll([[1, 2], [3, 4]], [[1, 2], [1, 2]])` returns `1`. + ## indexOf(arr, x) Returns the index of the first 'x' element (starting from 1) if it is in the array, or 0 if it is not. @@ -99,7 +172,7 @@ Returns the index of the first 'x' element (starting from 1) if it is in the arr Example: ``` -:) select indexOf([1,3,NULL,NULL],NULL) +:) SELECT indexOf([1,3,NULL,NULL],NULL) SELECT indexOf([1, 3, NULL, NULL], NULL) @@ -230,7 +303,7 @@ Removes the last item from the array. arrayPopBack(array) ``` -**Arguments** +**Parameters** - `array` – Array. @@ -254,7 +327,7 @@ Removes the first item from the array. arrayPopFront(array) ``` -**Arguments** +**Parameters** - `array` – Array. @@ -278,7 +351,7 @@ Adds one item to the end of the array. arrayPushBack(array, single_value) ``` -**Arguments** +**Parameters** - `array` – Array. - `single_value` – A single value. Only numbers can be added to an array with numbers, and only strings can be added to an array of strings. When adding numbers, ClickHouse automatically sets the `single_value` type for the data type of the array. For more information about the types of data in ClickHouse, see "[Data types](../../data_types/index.md#data_types)". Can be `NULL`. The function adds a `NULL` element to an array, and the type of array elements converts to `Nullable`. @@ -303,7 +376,7 @@ Adds one element to the beginning of the array. arrayPushFront(array, single_value) ``` -**Arguments** +**Parameters** - `array` – Array. - `single_value` – A single value. Only numbers can be added to an array with numbers, and only strings can be added to an array of strings. When adding numbers, ClickHouse automatically sets the `single_value` type for the data type of the array. For more information about the types of data in ClickHouse, see "[Data types](../../data_types/index.md#data_types)". Can be `NULL`. The function adds a `NULL` element to an array, and the type of array elements converts to `Nullable`. @@ -366,7 +439,7 @@ Returns a slice of the array. arraySlice(array, offset[, length]) ``` -**Arguments** +**Parameters** - `array` – Array of data. - `offset` – Indent from the edge of the array. A positive value indicates an offset on the left, and a negative value is an indent on the right. Numbering of the array items begins with 1. From 1c3c4c089050ce8ade57ad93b3c06ca963a2af71 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 22 Oct 2018 23:35:28 +0300 Subject: [PATCH 75/79] Update FunctionsArithmetic.h --- dbms/src/Functions/FunctionsArithmetic.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Functions/FunctionsArithmetic.h b/dbms/src/Functions/FunctionsArithmetic.h index 19495fc3ec4..ab0b5a1ec00 100644 --- a/dbms/src/Functions/FunctionsArithmetic.h +++ b/dbms/src/Functions/FunctionsArithmetic.h @@ -997,9 +997,9 @@ template <> constexpr bool IsIntegral = true; template <> constexpr bool IsIntegral = true; template <> constexpr bool IsIntegral = true; -template constexpr bool IsNonIntegral = false; -template <> constexpr bool IsNonIntegral = true; -template <> constexpr bool IsNonIntegral = true; +template constexpr bool IsFloatingPoint = false; +template <> constexpr bool IsFloatingPoint = true; +template <> constexpr bool IsFloatingPoint = true; template constexpr bool IsDateOrDateTime = false; template <> constexpr bool IsDateOrDateTime = true; @@ -1063,7 +1063,7 @@ public: /// Date % Int32 -> int32 Case>, Switch< Case && IsIntegral, RightDataType>, - Case && IsNonIntegral, DataTypeInt32>>>>; + Case && IsFloatingPoint, DataTypeInt32>>>>; }; From 25d9918b787b26a8116a4122369be0f8873e9612 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 22 Oct 2018 23:38:16 +0300 Subject: [PATCH 76/79] Update 00726_modulo_for_date.sql --- .../0_stateless/00726_modulo_for_date.sql | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00726_modulo_for_date.sql b/dbms/tests/queries/0_stateless/00726_modulo_for_date.sql index d7ae1e4545a..18f48ad447d 100644 --- a/dbms/tests/queries/0_stateless/00726_modulo_for_date.sql +++ b/dbms/tests/queries/0_stateless/00726_modulo_for_date.sql @@ -1,12 +1,13 @@ -SELECT toDate('21-06-2018') % 234 = toInt16(toDate('21-06-2018')) % 234; -SELECT toDate('21-06-2018') % 23456 = toInt16(toDate('21-06-2018')) % 23456; -SELECT toDate('21-06-2018') % 12376 = toInt16(toDate('21-06-2018')) % 12376; -SELECT toDate('21-06-2018 12:12:12') % 234 = toInt32(toDate('21-06-2018 12:12:12')) % 234; -SELECT toDate('21-06-2018 12:12:12') % 23456 = toInt32(toDate('21-06-2018 12:12:12')) % 23456; -SELECT toDate('21-06-2018 12:12:12') % 12376 = toInt32(toDate('21-06-2018 12:12:12')) % 12376; -SELECT toDate('21-06-2018') % 234.8 = toInt16(toDate('21-06-2018')) % 234.8; -SELECT toDate('21-06-2018') % 23456.8 = toInt16(toDate('21-06-2018')) % 23456.8; -SELECT toDate('21-06-2018') % 12376.8 = toInt16(toDate('21-06-2018')) % 12376.8; -SELECT toDate('21-06-2018 12:12:12') % 234.8 = toInt32(toDate('21-06-2018 12:12:12')) % 234.8; -SELECT toDate('21-06-2018 12:12:12') % 23456.8 = toInt32(toDate('21-06-2018 12:12:12')) % 23456.8; -SELECT toDate('21-06-2018 12:12:12') % 12376.8 = toInt32(toDate('21-06-2018 12:12:12')) % 12376.8; +SELECT toDate('2018-06-21') % 234 = toInt16(toDate('2018-06-21')) % 234; +SELECT toDate('2018-06-21') % 23456 = toInt16(toDate('2018-06-21')) % 23456; +SELECT toDate('2018-06-21') % 12376 = toInt16(toDate('2018-06-21')) % 12376; +SELECT toDateTime('2018-06-21 12:12:12') % 234 = toInt32(toDateTime('2018-06-21 12:12:12')) % 234; +SELECT toDateTime('2018-06-21 12:12:12') % 23456 = toInt32(toDateTime('2018-06-21 12:12:12')) % 23456; +SELECT toDateTime('2018-06-21 12:12:12') % 12376 = toInt32(toDateTime('2018-06-21 12:12:12')) % 12376; + +SELECT toDate('2018-06-21') % 234.8 = toInt16(toDate('2018-06-21')) % 234.8; +SELECT toDate('2018-06-21') % 23456.8 = toInt16(toDate('2018-06-21')) % 23456.8; +SELECT toDate('2018-06-21') % 12376.8 = toInt16(toDate('2018-06-21')) % 12376.8; +SELECT toDateTime('2018-06-21 12:12:12') % 234.8 = toInt32(toDateTime('2018-06-21 12:12:12')) % 234.8; +SELECT toDateTime('2018-06-21 12:12:12') % 23456.8 = toInt32(toDateTime('2018-06-21 12:12:12')) % 23456.8; +SELECT toDateTime('2018-06-21 12:12:12') % 12376.8 = toInt32(toDateTime('2018-06-21 12:12:12')) % 12376.8; From b8cfa77ea04a50346c626cba0b59902f50460fb2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Oct 2018 23:58:16 +0300 Subject: [PATCH 77/79] Fixed build [#CLICKHOUSE-2] --- dbms/src/Storages/StorageMaterializedView.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index e62db855079..c20757b802b 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -60,7 +60,6 @@ public: unsigned num_streams) override; String getDataPath() const override; - StoragePtr tryGetTargetTable() const; private: String select_database_name; @@ -73,7 +72,6 @@ private: Context & global_context; bool has_inner_table = false; - StoragePtr getTargetTable() const; void checkStatementCanBeForwarded() const; protected: From bac6b6f381c0aa486dfb8093cfaa4f190b24e737 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 23 Oct 2018 11:19:47 +0300 Subject: [PATCH 78/79] Fixed error #2863 --- dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h | 2 +- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 4d52acc8cbb..576b88f2e41 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -23,7 +23,7 @@ public: * max_blocks_number_to_read - if not nullptr, do not read all the parts whose right border is greater than max_block in partition. */ using PartitionIdToMaxBlock = std::unordered_map; - + BlockInputStreams read( const Names & column_names, const SelectQueryInfo & query_info, diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index f7eeaf97d94..1d29a6dba8b 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2675,7 +2675,7 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) Coordination::Requests ops; Coordination::Responses responses; - + Coordination::Stat added_parts_stat; String old_added_parts = zookeeper->get(quorum_last_part_path, &added_parts_stat); @@ -2686,7 +2686,7 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) auto part_info = MergeTreePartInfo::fromPartName(part_name, data.format_version); parts_with_quorum.added_parts[part_info.partition_id] = part_name; - + String new_added_parts = parts_with_quorum.toString(); ops.emplace_back(zkutil::makeRemoveRequest(quorum_status_path, stat.version)); From 21e7c25134e20458ad83537b401229900b559dd1 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 23 Oct 2018 11:32:28 +0300 Subject: [PATCH 79/79] Update FinishSortingBlockInputStream.cpp --- dbms/src/DataStreams/FinishSortingBlockInputStream.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp b/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp index 7a16737f928..015ea8bf903 100644 --- a/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp +++ b/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp @@ -64,7 +64,7 @@ struct Less Block FinishSortingBlockInputStream::readImpl() { - if (limit && total_rows_processed == limit) + if (limit && total_rows_processed >= limit) return {}; Block res;