From 6060a1ab575ac12f3f2be4f280c9c1f4ed24b1f7 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Fri, 4 Sep 2020 16:48:51 +0100 Subject: [PATCH 001/314] ALTER TABLE ... DROP|DETACH PART for ReplicatedMergeTree --- src/Parsers/ASTAlterQuery.cpp | 3 +- src/Parsers/ParserAlterQuery.cpp | 23 +++- src/Storages/PartitionCommands.cpp | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 100 ++++++++++++++++-- src/Storages/StorageReplicatedMergeTree.h | 7 +- .../01451_replicated_detach_part.reference | 9 ++ .../01451_replicated_detach_part.sql | 28 +++++ .../01452_replicated_drop_part.reference | 6 ++ .../01452_replicated_drop_part.sql | 25 +++++ 9 files changed, 185 insertions(+), 17 deletions(-) create mode 100644 tests/queries/0_stateless/01451_replicated_detach_part.reference create mode 100644 tests/queries/0_stateless/01451_replicated_detach_part.sql create mode 100644 tests/queries/0_stateless/01452_replicated_drop_part.reference create mode 100644 tests/queries/0_stateless/01452_replicated_drop_part.sql diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 8b23302a05c..72e422412f5 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -166,7 +166,8 @@ void ASTAlterCommand::formatImpl( } else if (type == ASTAlterCommand::DROP_PARTITION) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << (detach ? "DETACH" : "DROP") << " PARTITION " + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str + << (detach ? "DETACH" : "DROP") << (part ? " PART " : " PARTITION ") << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 3f22aff9cf5..84fb1ac7d05 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -51,13 +51,15 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_modify("MODIFY"); ParserKeyword s_attach_partition("ATTACH PARTITION"); + ParserKeyword s_attach_part("ATTACH PART"); ParserKeyword s_detach_partition("DETACH PARTITION"); + ParserKeyword s_detach_part("DETACH PART"); ParserKeyword s_drop_partition("DROP PARTITION"); + ParserKeyword s_drop_part("DROP PART"); ParserKeyword s_move_partition("MOVE PARTITION"); + ParserKeyword s_move_part("MOVE PART"); ParserKeyword s_drop_detached_partition("DROP DETACHED PARTITION"); ParserKeyword s_drop_detached_part("DROP DETACHED PART"); - ParserKeyword s_attach_part("ATTACH PART"); - ParserKeyword s_move_part("MOVE PART"); ParserKeyword s_fetch_partition("FETCH PARTITION"); ParserKeyword s_replace_partition("REPLACE PARTITION"); ParserKeyword s_freeze("FREEZE"); @@ -149,6 +151,14 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->type = ASTAlterCommand::DROP_PARTITION; } + else if (s_drop_part.ignore(pos, expected)) + { + if (!parser_string_literal.parse(pos, command->partition, expected)) + return false; + + command->type = ASTAlterCommand::DROP_PARTITION; + command->part = true; + } else if (s_drop_detached_partition.ignore(pos, expected)) { if (!parser_partition.parse(pos, command->partition, expected)) @@ -342,6 +352,15 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->type = ASTAlterCommand::DROP_PARTITION; command->detach = true; } + else if (s_detach_part.ignore(pos, expected)) + { + if (!parser_string_literal.parse(pos, command->partition, expected)) + return false; + + command->type = ASTAlterCommand::DROP_PARTITION; + command->part = true; + command->detach = true; + } else if (s_attach_partition.ignore(pos, expected)) { if (!parser_partition.parse(pos, command->partition, expected)) diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index b21d22e5e1a..76c2af17256 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -21,6 +21,7 @@ std::optional PartitionCommand::parse(const ASTAlterCommand * res.type = DROP_PARTITION; res.partition = command_ast->partition; res.detach = command_ast->detach; + res.part = command_ast->part; return res; } else if (command_ast->type == ASTAlterCommand::DROP_DETACHED_PARTITION) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index cac2219b062..312040cbf16 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3894,8 +3894,13 @@ Pipe StorageReplicatedMergeTree::alterPartition( switch (command.type) { case PartitionCommand::DROP_PARTITION: - checkPartitionCanBeDropped(command.partition); - dropPartition(query, command.partition, command.detach, query_context); + if (command.part) + { + /// TODO(nv) what to check here? it is possible to drop a big partition by dropping small parts... + } + else + checkPartitionCanBeDropped(command.partition); + dropPartition(query, command.partition, command.detach, command.part, query_context); break; case PartitionCommand::DROP_DETACHED_PARTITION: @@ -4017,18 +4022,30 @@ bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const St } -void StorageReplicatedMergeTree::dropPartition(const ASTPtr &, const ASTPtr & partition, bool detach, const Context & query_context) +void StorageReplicatedMergeTree::dropPartition( + const ASTPtr &, const ASTPtr & partition, bool detach, bool drop_part, const Context & query_context) { assertNotReadonly(); if (!is_leader) - throw Exception("DROP PARTITION cannot be done on this replica because it is not a leader", ErrorCodes::NOT_A_LEADER); + throw Exception("DROP PART|PARTITION cannot be done on this replica because it is not a leader", ErrorCodes::NOT_A_LEADER); zkutil::ZooKeeperPtr zookeeper = getZooKeeper(); - String partition_id = getPartitionIDFromQuery(partition, query_context); - LogEntry entry; - if (dropPartsInPartition(*zookeeper, partition_id, entry, detach)) + bool did_drop; + + if (drop_part) + { + String part_name = partition->as().value.safeGet(); + did_drop = dropPart(zookeeper, part_name, entry, detach); + } + else + { + String partition_id = getPartitionIDFromQuery(partition, query_context); + did_drop = dropAllPartsInPartition(*zookeeper, partition_id, entry, detach); + } + + if (did_drop) { /// If necessary, wait until the operation is performed on itself or on all replicas. if (query_context.getSettingsRef().replication_alter_partitions_sync != 0) @@ -4041,7 +4058,12 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr &, const ASTPtr & pa } /// Cleaning possibly stored information about parts from /quorum/last_part node in ZooKeeper. - cleanLastPartNode(partition_id); + /// TODO(nv) how is this related to dropPart? Is it? + if (!drop_part) + { + String partition_id = getPartitionIDFromQuery(partition, query_context); + cleanLastPartNode(partition_id); + } } @@ -4062,7 +4084,7 @@ void StorageReplicatedMergeTree::truncate( { LogEntry entry; - if (dropPartsInPartition(*zookeeper, partition_id, entry, false)) + if (dropAllPartsInPartition(*zookeeper, partition_id, entry, false)) waitForAllReplicasToProcessLogEntry(entry); } } @@ -5665,9 +5687,65 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI return true; } +bool StorageReplicatedMergeTree::dropPart( + zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach) +{ + LOG_TRACE(log, "Will try to insert a log entry to DROP_RANGE for part: " + part_name); -bool StorageReplicatedMergeTree::dropPartsInPartition( - zkutil::ZooKeeper & zookeeper, String & partition_id, StorageReplicatedMergeTree::LogEntry & entry, bool detach) + auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); + + while (true) + { + ReplicatedMergeTreeMergePredicate merge_pred = queue.getMergePredicate(zookeeper); + + /// TODO(nv) It is possible that part does not exist on replica which executes this statement. + /// Also, it possible for the part to not exist on any replicas, replica which created log entries for the part disappeared. + auto part = data_parts_by_info.find(part_info); + if (part == data_parts_by_info.end()) + throw Exception("Part " + part_name + " not found locally, won't try to drop it.", ErrorCodes::NOT_IMPLEMENTED); + + /// TODO(nv) get ops and commit together w/ log entry? + clearBlocksInPartition(*zookeeper, part_info.partition_id, part_info.min_block, part_info.max_block); + + /// There isn't a lot we can do otherwise. Can't cancel merges because it is possible that a replica already + /// finished the merge. + if (partIsAssignedToBackgroundOperation(*part)) + throw Exception("Part " + part_name + + " is currently participating in a background operation (mutation/merge)" + + ", try again later.", ErrorCodes::PART_IS_TEMPORARILY_LOCKED); + + /// If `part_name` is result of a recent merge and source parts are still available then + /// DROP_RANGE with detach will move this part together with source parts to `detached/` dir. + entry.type = LogEntry::DROP_RANGE; + entry.source_replica = replica_name; + entry.new_part_name = part_name; + entry.detach = detach; + entry.create_time = time(nullptr); + + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/log", merge_pred.getVersion())); /// Make sure no new events were added to the log. + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); + ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); /// Just update version. + Coordination::Responses responses; + Coordination::Error rc = zookeeper->tryMulti(ops, responses); + + if (rc == Coordination::Error::ZBADVERSION) + { + LOG_TRACE(log, "A new log entry appeared while trying to commit DROP RANGE. Retry."); + continue; + } + else + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path = dynamic_cast(*responses[1]).path_created; + entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + return true; + } +} + +bool StorageReplicatedMergeTree::dropAllPartsInPartition( + zkutil::ZooKeeper & zookeeper, String & partition_id, LogEntry & entry, bool detach) { MergeTreePartInfo drop_range_info; if (!getFakePartCoveringAllPartsInPartition(partition_id, drop_range_info)) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index ad992a10f08..ece2294ac60 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -529,11 +529,12 @@ private: /// Info about how other replicas can access this one. ReplicatedMergeTreeAddress getReplicatedMergeTreeAddress() const; - bool dropPartsInPartition(zkutil::ZooKeeper & zookeeper, String & partition_id, - StorageReplicatedMergeTree::LogEntry & entry, bool detach); + bool dropPart(zkutil::ZooKeeperPtr & zookeeper, String partition_id, LogEntry & entry, bool detach); + bool dropAllPartsInPartition( + zkutil::ZooKeeper & zookeeper, String & partition_id, LogEntry & entry, bool detach); // Partition helpers - void dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & query_context); + void dropPartition(const ASTPtr &, const ASTPtr & partition, bool detach, bool drop_part, const Context & query_context); PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, const Context & query_context); void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & query_context); void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & query_context); diff --git a/tests/queries/0_stateless/01451_replicated_detach_part.reference b/tests/queries/0_stateless/01451_replicated_detach_part.reference new file mode 100644 index 00000000000..b68ee092c91 --- /dev/null +++ b/tests/queries/0_stateless/01451_replicated_detach_part.reference @@ -0,0 +1,9 @@ +0 +1 +2 +0 +2 +all_1_1_0 +0 +1 +2 diff --git a/tests/queries/0_stateless/01451_replicated_detach_part.sql b/tests/queries/0_stateless/01451_replicated_detach_part.sql new file mode 100644 index 00000000000..602fca2d371 --- /dev/null +++ b/tests/queries/0_stateless/01451_replicated_detach_part.sql @@ -0,0 +1,28 @@ +SET replication_alter_partitions_sync = 2; + +DROP TABLE IF EXISTS attach_01451_r1; +DROP TABLE IF EXISTS attach_01451_r2; + +CREATE TABLE attach_01451_r1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01451/attach', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; +CREATE TABLE attach_01451_r2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01451/attach', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; + +INSERT INTO attach_01451_r1 VALUES (0); +INSERT INTO attach_01451_r1 VALUES (1); +INSERT INTO attach_01451_r1 VALUES (2); + +SELECT v FROM attach_01451_r1 ORDER BY v; + +ALTER TABLE attach_01451_r2 DETACH PART 'all_1_1_0'; + +SELECT v FROM attach_01451_r1 ORDER BY v; + +SELECT name FROM system.detached_parts WHERE table = 'attach_01451_r2'; + +ALTER TABLE attach_01451_r2 ATTACH PART 'all_1_1_0'; + +SELECT v FROM attach_01451_r1 ORDER BY v; + +SELECT name FROM system.detached_parts WHERE table = 'attach_01451_r2'; + +DROP TABLE attach_01451_r1; +DROP TABLE attach_01451_r2; diff --git a/tests/queries/0_stateless/01452_replicated_drop_part.reference b/tests/queries/0_stateless/01452_replicated_drop_part.reference new file mode 100644 index 00000000000..6fd8504e46b --- /dev/null +++ b/tests/queries/0_stateless/01452_replicated_drop_part.reference @@ -0,0 +1,6 @@ +0 +1 +2 +0 +2 +all_0_2_1 diff --git a/tests/queries/0_stateless/01452_replicated_drop_part.sql b/tests/queries/0_stateless/01452_replicated_drop_part.sql new file mode 100644 index 00000000000..406019fdaf1 --- /dev/null +++ b/tests/queries/0_stateless/01452_replicated_drop_part.sql @@ -0,0 +1,25 @@ +SET replication_alter_partitions_sync = 2; + +DROP TABLE IF EXISTS attach_r1; +DROP TABLE IF EXISTS attach_r2; + +CREATE TABLE attach_r1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01452/attach', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; +CREATE TABLE attach_r2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01452/attach', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; + +INSERT INTO attach_r1 VALUES (0); +INSERT INTO attach_r1 VALUES (1); +INSERT INTO attach_r1 VALUES (2); + +SELECT v FROM attach_r1 ORDER BY v; + +ALTER TABLE attach_r2 DROP PART 'all_1_1_0'; + +SELECT v FROM attach_r1 ORDER BY v; + +ALTER TABLE attach_r1 MODIFY SETTING max_replicated_merges_in_queue = 1; +OPTIMIZE TABLE attach_r1 FINAL; + +SELECT name FROM system.parts WHERE table = 'attach_r1' AND active; + +DROP TABLE attach_r1; +DROP TABLE attach_r2; From 0d28fdd1161511ac9d1eb56c6291f7fba58a2d4b Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Mon, 7 Sep 2020 13:58:34 +0100 Subject: [PATCH 002/314] Sync replicas between operations --- tests/queries/0_stateless/01451_replicated_detach_part.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01451_replicated_detach_part.sql b/tests/queries/0_stateless/01451_replicated_detach_part.sql index 602fca2d371..b414bb09e69 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_part.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_part.sql @@ -12,6 +12,7 @@ INSERT INTO attach_01451_r1 VALUES (2); SELECT v FROM attach_01451_r1 ORDER BY v; +SYSTEM SYNC REPLICA attach_01451_r2; ALTER TABLE attach_01451_r2 DETACH PART 'all_1_1_0'; SELECT v FROM attach_01451_r1 ORDER BY v; @@ -20,6 +21,7 @@ SELECT name FROM system.detached_parts WHERE table = 'attach_01451_r2'; ALTER TABLE attach_01451_r2 ATTACH PART 'all_1_1_0'; +SYSTEM SYNC REPLICA attach_01451_r1; SELECT v FROM attach_01451_r1 ORDER BY v; SELECT name FROM system.detached_parts WHERE table = 'attach_01451_r2'; From 18763747705f214918cc1c4b0d8623dde9e1144d Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Mon, 7 Sep 2020 16:05:19 +0100 Subject: [PATCH 003/314] ALTER TABLE ... DROP|DETACH PART for MergeTree --- src/Storages/StorageMergeTree.cpp | 32 +++++++++++++++---- src/Storages/StorageMergeTree.h | 2 +- .../01451_detach_drop_part.reference | 12 +++++++ .../0_stateless/01451_detach_drop_part.sql | 31 ++++++++++++++++++ 4 files changed, 69 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/01451_detach_drop_part.reference create mode 100644 tests/queries/0_stateless/01451_detach_drop_part.sql diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 8b233fb2408..9ea83b3bf73 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1066,8 +1066,13 @@ Pipe StorageMergeTree::alterPartition( switch (command.type) { case PartitionCommand::DROP_PARTITION: - checkPartitionCanBeDropped(command.partition); - dropPartition(command.partition, command.detach, query_context); + if (command.part) + { + /// TODO(nv) what would be a good check here? + } + else + checkPartitionCanBeDropped(command.partition); + dropPartition(command.partition, command.detach, command.part, query_context); break; case PartitionCommand::DROP_DETACHED_PARTITION: @@ -1165,7 +1170,7 @@ ActionLock StorageMergeTree::stopMergesAndWait() } -void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, const Context & context) +void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & context) { { /// Asks to complete merges and does not allow them to start. @@ -1173,10 +1178,23 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons auto merge_blocker = stopMergesAndWait(); auto metadata_snapshot = getInMemoryMetadataPtr(); - String partition_id = getPartitionIDFromQuery(partition, context); + + MergeTreeData::DataPartsVector parts_to_remove; /// TODO: should we include PreComitted parts like in Replicated case? - auto parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + if (drop_part) + { + String part_name = partition->as().value.safeGet(); + auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Committed}); + + if (part) + parts_to_remove.push_back(part); + } else + { + String partition_id = getPartitionIDFromQuery(partition, context); + parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + } + // TODO should we throw an exception if parts_to_remove is empty? removePartsFromWorkingSet(parts_to_remove, true); @@ -1191,9 +1209,9 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons } if (detach) - LOG_INFO(log, "Detached {} parts inside partition ID {}.", parts_to_remove.size(), partition_id); + LOG_INFO(log, "Detached {} parts.", parts_to_remove.size()); else - LOG_INFO(log, "Removed {} parts inside partition ID {}.", parts_to_remove.size(), partition_id); + LOG_INFO(log, "Removed {} parts.", parts_to_remove.size()); } clearOldPartsFromFilesystem(); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 5662f9e0088..2f82f88e7ca 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -150,7 +150,7 @@ private: void clearOldMutations(bool truncate = false); // Partition helpers - void dropPartition(const ASTPtr & partition, bool detach, const Context & context); + void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & context); PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, bool part, const Context & context); void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context); diff --git a/tests/queries/0_stateless/01451_detach_drop_part.reference b/tests/queries/0_stateless/01451_detach_drop_part.reference new file mode 100644 index 00000000000..48e639c9125 --- /dev/null +++ b/tests/queries/0_stateless/01451_detach_drop_part.reference @@ -0,0 +1,12 @@ +0 +1 +2 +0 +2 +all_2_2_0 +0 +1 +2 +-- drop part -- +0 +2 diff --git a/tests/queries/0_stateless/01451_detach_drop_part.sql b/tests/queries/0_stateless/01451_detach_drop_part.sql new file mode 100644 index 00000000000..92adaef14d9 --- /dev/null +++ b/tests/queries/0_stateless/01451_detach_drop_part.sql @@ -0,0 +1,31 @@ +DROP TABLE IF EXISTS attach_01451_mt; + +CREATE TABLE attach_01451_mt (v UInt8) ENGINE = MergeTree() order by tuple(); + +INSERT INTO attach_01451_mt VALUES (0); +INSERT INTO attach_01451_mt VALUES (1); +INSERT INTO attach_01451_mt VALUES (2); + +SELECT v FROM attach_01451_mt ORDER BY v; + +ALTER TABLE attach_01451_mt DETACH PART 'all_2_2_0'; + +SELECT v FROM attach_01451_mt ORDER BY v; + +SELECT name FROM system.detached_parts WHERE table = 'attach_01451_mt'; + +ALTER TABLE attach_01451_mt ATTACH PART 'all_2_2_0'; + +SELECT v FROM attach_01451_mt ORDER BY v; + +SELECT name FROM system.detached_parts WHERE table = 'attach_01451_mt'; + +SELECT '-- drop part --'; + +ALTER TABLE attach_01451_mt DROP PART 'all_4_4_0'; + +ALTER TABLE attach_01451_mt ATTACH PART 'all_4_4_0'; -- { serverError 233 } + +SELECT v FROM attach_01451_mt ORDER BY v; + +DROP TABLE attach_01451_mt; From 97d0b5ab235aa9e7c5c63ac83eba58e8a62136de Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Mon, 7 Sep 2020 16:19:47 +0100 Subject: [PATCH 004/314] Remove unused query argument --- src/Interpreters/InterpreterAlterQuery.cpp | 2 +- src/Storages/IStorage.cpp | 1 - src/Storages/IStorage.h | 1 - src/Storages/StorageMaterializedView.cpp | 4 ++-- src/Storages/StorageMaterializedView.h | 2 +- src/Storages/StorageMergeTree.cpp | 3 +-- src/Storages/StorageMergeTree.h | 1 - src/Storages/StorageReplicatedMergeTree.cpp | 6 ++---- src/Storages/StorageReplicatedMergeTree.h | 3 +-- 9 files changed, 8 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index c0ed5d8a36c..bf55e372066 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -87,7 +87,7 @@ BlockIO InterpreterAlterQuery::execute() if (!partition_commands.empty()) { table->checkAlterPartitionIsPossible(partition_commands, metadata_snapshot, context.getSettingsRef()); - auto partition_commands_pipe = table->alterPartition(query_ptr, metadata_snapshot, partition_commands, context); + auto partition_commands_pipe = table->alterPartition(metadata_snapshot, partition_commands, context); if (!partition_commands_pipe.empty()) res.pipeline.init(std::move(partition_commands_pipe)); } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 494b6a19c02..1eea364190e 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -92,7 +92,6 @@ Pipe IStorage::read( } Pipe IStorage::alterPartition( - const ASTPtr & /* query */, const StorageMetadataPtr & /* metadata_snapshot */, const PartitionCommands & /* commands */, const Context & /* context */) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index bdbb2eb9d80..d874663ae11 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -348,7 +348,6 @@ public: * Should handle locks for each command on its own. */ virtual Pipe alterPartition( - const ASTPtr & /* query */, const StorageMetadataPtr & /* metadata_snapshot */, const PartitionCommands & /* commands */, const Context & /* context */); diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index a2e3fae0951..fd2cec95647 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -249,10 +249,10 @@ void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & command } Pipe StorageMaterializedView::alterPartition( - const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) + const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) { checkStatementCanBeForwarded(); - return getTargetTable()->alterPartition(query, metadata_snapshot, commands, context); + return getTargetTable()->alterPartition(metadata_snapshot, commands, context); } void StorageMaterializedView::checkAlterPartitionIsPossible( diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 1ee4246c7f1..4faa3b69fe7 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -51,7 +51,7 @@ public: void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override; - Pipe alterPartition(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) override; + Pipe alterPartition(const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) override; void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9ea83b3bf73..8164bce0f4d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1054,7 +1054,6 @@ bool StorageMergeTree::optimize( } Pipe StorageMergeTree::alterPartition( - const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & query_context) @@ -1130,7 +1129,7 @@ Pipe StorageMergeTree::alterPartition( break; default: - IStorage::alterPartition(query, metadata_snapshot, commands, query_context); // should throw an exception. + IStorage::alterPartition(metadata_snapshot, commands, query_context); // should throw an exception. } for (auto & command_result : current_command_results) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 2f82f88e7ca..8c970e56f13 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -62,7 +62,6 @@ public: const Context & context) override; Pipe alterPartition( - const ASTPtr & query, const StorageMetadataPtr & /* metadata_snapshot */, const PartitionCommands & commands, const Context & context) override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 312040cbf16..e8007440f2f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3882,7 +3882,6 @@ void StorageReplicatedMergeTree::alter( } Pipe StorageReplicatedMergeTree::alterPartition( - const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & query_context) @@ -3900,7 +3899,7 @@ Pipe StorageReplicatedMergeTree::alterPartition( } else checkPartitionCanBeDropped(command.partition); - dropPartition(query, command.partition, command.detach, command.part, query_context); + dropPartition(command.partition, command.detach, command.part, query_context); break; case PartitionCommand::DROP_DETACHED_PARTITION: @@ -4022,8 +4021,7 @@ bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const St } -void StorageReplicatedMergeTree::dropPartition( - const ASTPtr &, const ASTPtr & partition, bool detach, bool drop_part, const Context & query_context) +void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & query_context) { assertNotReadonly(); if (!is_leader) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index ece2294ac60..58c82511a27 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -112,7 +112,6 @@ public: void alter(const AlterCommands & params, const Context & query_context, TableLockHolder & table_lock_holder) override; Pipe alterPartition( - const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & query_context) override; @@ -534,7 +533,7 @@ private: zkutil::ZooKeeper & zookeeper, String & partition_id, LogEntry & entry, bool detach); // Partition helpers - void dropPartition(const ASTPtr &, const ASTPtr & partition, bool detach, bool drop_part, const Context & query_context); + void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & query_context); PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, const Context & query_context); void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & query_context); void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & query_context); From 1007d2676197a770ddd352a1e9e86699e44f1a7e Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Mon, 7 Sep 2020 17:23:59 +0100 Subject: [PATCH 005/314] Simplify ALTER TABLE ... DETACH|DROP PART tests --- .../01451_detach_drop_part.reference | 4 ++ .../0_stateless/01451_detach_drop_part.sql | 41 +++++++++------- ...1451_replicated_detach_drop_part.reference | 16 +++++++ .../01451_replicated_detach_drop_part.sql | 47 +++++++++++++++++++ .../01451_replicated_detach_part.reference | 9 ---- .../01451_replicated_detach_part.sql | 30 ------------ .../01452_replicated_drop_part.reference | 6 --- .../01452_replicated_drop_part.sql | 25 ---------- 8 files changed, 92 insertions(+), 86 deletions(-) create mode 100644 tests/queries/0_stateless/01451_replicated_detach_drop_part.reference create mode 100644 tests/queries/0_stateless/01451_replicated_detach_drop_part.sql delete mode 100644 tests/queries/0_stateless/01451_replicated_detach_part.reference delete mode 100644 tests/queries/0_stateless/01451_replicated_detach_part.sql delete mode 100644 tests/queries/0_stateless/01452_replicated_drop_part.reference delete mode 100644 tests/queries/0_stateless/01452_replicated_drop_part.sql diff --git a/tests/queries/0_stateless/01451_detach_drop_part.reference b/tests/queries/0_stateless/01451_detach_drop_part.reference index 48e639c9125..bc4f1b6be80 100644 --- a/tests/queries/0_stateless/01451_detach_drop_part.reference +++ b/tests/queries/0_stateless/01451_detach_drop_part.reference @@ -10,3 +10,7 @@ all_2_2_0 -- drop part -- 0 2 +-- resume merges -- +0 +2 +all_1_3_1 diff --git a/tests/queries/0_stateless/01451_detach_drop_part.sql b/tests/queries/0_stateless/01451_detach_drop_part.sql index 92adaef14d9..47e61f2d924 100644 --- a/tests/queries/0_stateless/01451_detach_drop_part.sql +++ b/tests/queries/0_stateless/01451_detach_drop_part.sql @@ -1,31 +1,40 @@ -DROP TABLE IF EXISTS attach_01451_mt; +DROP TABLE IF EXISTS mt; -CREATE TABLE attach_01451_mt (v UInt8) ENGINE = MergeTree() order by tuple(); +CREATE TABLE mt (v UInt8) ENGINE = MergeTree() order by tuple(); +SYSTEM STOP MERGES; -INSERT INTO attach_01451_mt VALUES (0); -INSERT INTO attach_01451_mt VALUES (1); -INSERT INTO attach_01451_mt VALUES (2); +INSERT INTO mt VALUES (0); +INSERT INTO mt VALUES (1); +INSERT INTO mt VALUES (2); -SELECT v FROM attach_01451_mt ORDER BY v; +SELECT v FROM mt ORDER BY v; -ALTER TABLE attach_01451_mt DETACH PART 'all_2_2_0'; +ALTER TABLE mt DETACH PART 'all_2_2_0'; -SELECT v FROM attach_01451_mt ORDER BY v; +SELECT v FROM mt ORDER BY v; -SELECT name FROM system.detached_parts WHERE table = 'attach_01451_mt'; +SELECT name FROM system.detached_parts WHERE table = 'mt'; -ALTER TABLE attach_01451_mt ATTACH PART 'all_2_2_0'; +ALTER TABLE mt ATTACH PART 'all_2_2_0'; -SELECT v FROM attach_01451_mt ORDER BY v; +SELECT v FROM mt ORDER BY v; -SELECT name FROM system.detached_parts WHERE table = 'attach_01451_mt'; +SELECT name FROM system.detached_parts WHERE table = 'mt'; SELECT '-- drop part --'; -ALTER TABLE attach_01451_mt DROP PART 'all_4_4_0'; +ALTER TABLE mt DROP PART 'all_4_4_0'; -ALTER TABLE attach_01451_mt ATTACH PART 'all_4_4_0'; -- { serverError 233 } +ALTER TABLE mt ATTACH PART 'all_4_4_0'; -- { serverError 233 } -SELECT v FROM attach_01451_mt ORDER BY v; +SELECT v FROM mt ORDER BY v; -DROP TABLE attach_01451_mt; +SELECT '-- resume merges --'; +SYSTEM START MERGES; +OPTIMIZE TABLE mt FINAL; + +SELECT v FROM mt ORDER BY v; + +SELECT name FROM system.parts WHERE table = 'mt' AND active; + +DROP TABLE mt; diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_part.reference b/tests/queries/0_stateless/01451_replicated_detach_drop_part.reference new file mode 100644 index 00000000000..109012cba70 --- /dev/null +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_part.reference @@ -0,0 +1,16 @@ +0 +1 +2 +0 +2 +all_1_1_0 +0 +1 +2 +-- drop part -- +0 +2 +-- resume merges -- +0 +2 +all_0_2_1 diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_part.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_part.sql new file mode 100644 index 00000000000..1209f11b68e --- /dev/null +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_part.sql @@ -0,0 +1,47 @@ +SET replication_alter_partitions_sync = 2; + +DROP TABLE IF EXISTS replica1; +DROP TABLE IF EXISTS replica2; + +CREATE TABLE replica1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01451/attach', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; +CREATE TABLE replica2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01451/attach', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; + +INSERT INTO replica1 VALUES (0); +INSERT INTO replica1 VALUES (1); +INSERT INTO replica1 VALUES (2); + +SELECT v FROM replica1 ORDER BY v; + +SYSTEM SYNC REPLICA replica2; +ALTER TABLE replica2 DETACH PART 'all_1_1_0'; + +SELECT v FROM replica1 ORDER BY v; + +SELECT name FROM system.detached_parts WHERE table = 'replica2'; + +ALTER TABLE replica2 ATTACH PART 'all_1_1_0'; + +SYSTEM SYNC REPLICA replica1; +SELECT v FROM replica1 ORDER BY v; + +SELECT name FROM system.detached_parts WHERE table = 'replica2'; + +SELECT '-- drop part --'; + +ALTER TABLE replica1 DROP PART 'all_3_3_0'; + +ALTER TABLE replica1 ATTACH PART 'all_3_3_0'; -- { serverError 233 } + +SELECT v FROM replica1 ORDER BY v; + +SELECT '-- resume merges --'; + +ALTER TABLE replica1 MODIFY SETTING max_replicated_merges_in_queue = 1; +OPTIMIZE TABLE replica1 FINAL; + +SELECT v FROM replica1 ORDER BY v; + +SELECT name FROM system.parts WHERE table = 'replica2' AND active; + +DROP TABLE replica1; +DROP TABLE replica2; diff --git a/tests/queries/0_stateless/01451_replicated_detach_part.reference b/tests/queries/0_stateless/01451_replicated_detach_part.reference deleted file mode 100644 index b68ee092c91..00000000000 --- a/tests/queries/0_stateless/01451_replicated_detach_part.reference +++ /dev/null @@ -1,9 +0,0 @@ -0 -1 -2 -0 -2 -all_1_1_0 -0 -1 -2 diff --git a/tests/queries/0_stateless/01451_replicated_detach_part.sql b/tests/queries/0_stateless/01451_replicated_detach_part.sql deleted file mode 100644 index b414bb09e69..00000000000 --- a/tests/queries/0_stateless/01451_replicated_detach_part.sql +++ /dev/null @@ -1,30 +0,0 @@ -SET replication_alter_partitions_sync = 2; - -DROP TABLE IF EXISTS attach_01451_r1; -DROP TABLE IF EXISTS attach_01451_r2; - -CREATE TABLE attach_01451_r1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01451/attach', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; -CREATE TABLE attach_01451_r2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01451/attach', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; - -INSERT INTO attach_01451_r1 VALUES (0); -INSERT INTO attach_01451_r1 VALUES (1); -INSERT INTO attach_01451_r1 VALUES (2); - -SELECT v FROM attach_01451_r1 ORDER BY v; - -SYSTEM SYNC REPLICA attach_01451_r2; -ALTER TABLE attach_01451_r2 DETACH PART 'all_1_1_0'; - -SELECT v FROM attach_01451_r1 ORDER BY v; - -SELECT name FROM system.detached_parts WHERE table = 'attach_01451_r2'; - -ALTER TABLE attach_01451_r2 ATTACH PART 'all_1_1_0'; - -SYSTEM SYNC REPLICA attach_01451_r1; -SELECT v FROM attach_01451_r1 ORDER BY v; - -SELECT name FROM system.detached_parts WHERE table = 'attach_01451_r2'; - -DROP TABLE attach_01451_r1; -DROP TABLE attach_01451_r2; diff --git a/tests/queries/0_stateless/01452_replicated_drop_part.reference b/tests/queries/0_stateless/01452_replicated_drop_part.reference deleted file mode 100644 index 6fd8504e46b..00000000000 --- a/tests/queries/0_stateless/01452_replicated_drop_part.reference +++ /dev/null @@ -1,6 +0,0 @@ -0 -1 -2 -0 -2 -all_0_2_1 diff --git a/tests/queries/0_stateless/01452_replicated_drop_part.sql b/tests/queries/0_stateless/01452_replicated_drop_part.sql deleted file mode 100644 index 406019fdaf1..00000000000 --- a/tests/queries/0_stateless/01452_replicated_drop_part.sql +++ /dev/null @@ -1,25 +0,0 @@ -SET replication_alter_partitions_sync = 2; - -DROP TABLE IF EXISTS attach_r1; -DROP TABLE IF EXISTS attach_r2; - -CREATE TABLE attach_r1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01452/attach', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; -CREATE TABLE attach_r2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01452/attach', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; - -INSERT INTO attach_r1 VALUES (0); -INSERT INTO attach_r1 VALUES (1); -INSERT INTO attach_r1 VALUES (2); - -SELECT v FROM attach_r1 ORDER BY v; - -ALTER TABLE attach_r2 DROP PART 'all_1_1_0'; - -SELECT v FROM attach_r1 ORDER BY v; - -ALTER TABLE attach_r1 MODIFY SETTING max_replicated_merges_in_queue = 1; -OPTIMIZE TABLE attach_r1 FINAL; - -SELECT name FROM system.parts WHERE table = 'attach_r1' AND active; - -DROP TABLE attach_r1; -DROP TABLE attach_r2; From a62e96c7aa03e556465f9d8418fe3de76720f9f0 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Mon, 7 Sep 2020 18:31:07 +0100 Subject: [PATCH 006/314] nit: fix argument names for clang-10 --- src/Storages/StorageReplicatedMergeTree.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 58c82511a27..3ce946a54c0 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -528,7 +528,7 @@ private: /// Info about how other replicas can access this one. ReplicatedMergeTreeAddress getReplicatedMergeTreeAddress() const; - bool dropPart(zkutil::ZooKeeperPtr & zookeeper, String partition_id, LogEntry & entry, bool detach); + bool dropPart(zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach); bool dropAllPartsInPartition( zkutil::ZooKeeper & zookeeper, String & partition_id, LogEntry & entry, bool detach); From cbedd44a60a6899d4a0c2e8f48ae502b1e2079b7 Mon Sep 17 00:00:00 2001 From: "philip.han" Date: Fri, 18 Sep 2020 17:43:07 +0900 Subject: [PATCH 007/314] Fix indexOf() to use BloomFilter --- .../MergeTreeIndexConditionBloomFilter.cpp | 33 +++++++++++++++---- .../MergeTreeIndexConditionBloomFilter.h | 2 ++ tests/performance/bloom_filter.xml | 10 ++++-- 3 files changed, 37 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index d4736444242..3a7f7408f7c 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -222,9 +222,21 @@ bool MergeTreeIndexConditionBloomFilter::traverseAtomAST(const ASTPtr & node, Bl } } + return traverseFunction(node, block_with_constants, out); +} + +bool MergeTreeIndexConditionBloomFilter::traverseFunction(const ASTPtr & node, Block & block_with_constants, RPNElement & out) +{ + bool maybe_useful = false; + if (const auto * function = node->as()) { const ASTs & arguments = function->arguments->children; + for (auto arg : arguments) + { + if (traverseFunction(arg, block_with_constants, out)) + maybe_useful = true; + } if (arguments.size() != 2) return false; @@ -232,20 +244,29 @@ bool MergeTreeIndexConditionBloomFilter::traverseAtomAST(const ASTPtr & node, Bl if (functionIsInOrGlobalInOperator(function->name)) { if (const auto & prepared_set = getPreparedSet(arguments[1])) - return traverseASTIn(function->name, arguments[0], prepared_set, out); + { + if (traverseASTIn(function->name, arguments[0], prepared_set, out)) + maybe_useful = true; + } } - else if (function->name == "equals" || function->name == "notEquals" || function->name == "has") + else if (function->name == "equals" || function->name == "notEquals" || function->name == "has" || function->name == "indexOf") { Field const_value; DataTypePtr const_type; if (KeyCondition::getConstant(arguments[1], block_with_constants, const_value, const_type)) - return traverseASTEquals(function->name, arguments[0], const_type, const_value, out); + { + if (traverseASTEquals(function->name, arguments[0], const_type, const_value, out)) + maybe_useful = true; + } else if (KeyCondition::getConstant(arguments[0], block_with_constants, const_value, const_type)) - return traverseASTEquals(function->name, arguments[1], const_type, const_value, out); + { + if (traverseASTEquals(function->name, arguments[1], const_type, const_value, out)) + maybe_useful = true; + } } } - return false; + return maybe_useful; } bool MergeTreeIndexConditionBloomFilter::traverseASTIn( @@ -311,7 +332,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals( const DataTypePtr & index_type = header.getByPosition(position).type; const auto * array_type = typeid_cast(index_type.get()); - if (function_name == "has") + if (function_name == "has" || function_name == "indexOf") { out.function = RPNElement::FUNCTION_HAS; diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h index 41b7416253f..40299cb7a43 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h @@ -67,6 +67,8 @@ private: bool traverseAtomAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out); + bool traverseFunction(const ASTPtr & node, Block & block_with_constants, RPNElement & out); + bool traverseASTIn(const String & function_name, const ASTPtr & key_ast, const SetPtr & prepared_set, RPNElement & out); bool traverseASTIn( diff --git a/tests/performance/bloom_filter.xml b/tests/performance/bloom_filter.xml index 3d9096afb03..d1881cde56e 100644 --- a/tests/performance/bloom_filter.xml +++ b/tests/performance/bloom_filter.xml @@ -1,10 +1,16 @@ - - DROP TABLE IF EXISTS test_bf CREATE TABLE test_bf (`id` int, `ary` Array(String), INDEX idx_ary ary TYPE bloom_filter(0.01) GRANULARITY 8192) ENGINE = MergeTree() ORDER BY id + CREATE TABLE test_bf_indexOf (`id` int, `ary` Array(String), INDEX idx_ary ary TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY id SYSTEM STOP MERGES + INSERT INTO test_bf SELECT number AS id, [CAST(id, 'String'), CAST(id + 1, 'String'), CAST(id + 2, 'String')] FROM numbers(1000000) + INSERT INTO test_bf_indexOf SELECT number AS id, [CAST(id, 'String'), CAST(id + 1, 'String'), CAST(id + 2, 'String')] FROM numbers(1000000) + + SELECT count() FROM test_bf_indexOf WHERE indexOf(ary, '1') > 0 + SELECT count() FROM test_bf_indexOf WHERE ary[indexOf(ary, '1')] = '1' + SYSTEM START MERGES + DROP TABLE IF EXISTS test_bf_indexOf DROP TABLE IF EXISTS test_bf From 322e63971d4592ec7769975f6830a1343e66b687 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 18 Sep 2020 21:47:32 +0300 Subject: [PATCH 008/314] Update MergeTreeIndexConditionBloomFilter.cpp --- src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 3a7f7408f7c..b78ed3adf45 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -232,7 +232,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const ASTPtr & node, B if (const auto * function = node->as()) { const ASTs & arguments = function->arguments->children; - for (auto arg : arguments) + for (const auto & arg : arguments) { if (traverseFunction(arg, block_with_constants, out)) maybe_useful = true; From 3122455b4d1c26dfe5832aaa99429fa677d21197 Mon Sep 17 00:00:00 2001 From: "philip.han" Date: Sun, 20 Sep 2020 19:04:33 +0900 Subject: [PATCH 009/314] BloomFilter is used just in specific cases of indexof() --- .../MergeTreeIndexConditionBloomFilter.cpp | 73 +++++++++++++++---- .../MergeTreeIndexConditionBloomFilter.h | 4 +- .../00945_bloom_filter_index.reference | 36 +++++++++ .../0_stateless/00945_bloom_filter_index.sql | 51 +++++++++++++ 4 files changed, 148 insertions(+), 16 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/00945_bloom_filter_index.sql diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index b78ed3adf45..69d63d0e80d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -222,10 +223,10 @@ bool MergeTreeIndexConditionBloomFilter::traverseAtomAST(const ASTPtr & node, Bl } } - return traverseFunction(node, block_with_constants, out); + return traverseFunction(node, block_with_constants, out, nullptr); } -bool MergeTreeIndexConditionBloomFilter::traverseFunction(const ASTPtr & node, Block & block_with_constants, RPNElement & out) +bool MergeTreeIndexConditionBloomFilter::traverseFunction(const ASTPtr & node, Block & block_with_constants, RPNElement & out, const ASTPtr & parent) { bool maybe_useful = false; @@ -234,7 +235,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const ASTPtr & node, B const ASTs & arguments = function->arguments->children; for (const auto & arg : arguments) { - if (traverseFunction(arg, block_with_constants, out)) + if (traverseFunction(arg, block_with_constants, out, node)) maybe_useful = true; } @@ -255,12 +256,12 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const ASTPtr & node, B DataTypePtr const_type; if (KeyCondition::getConstant(arguments[1], block_with_constants, const_value, const_type)) { - if (traverseASTEquals(function->name, arguments[0], const_type, const_value, out)) + if (traverseASTEquals(function->name, arguments[0], const_type, const_value, out, parent)) maybe_useful = true; } else if (KeyCondition::getConstant(arguments[0], block_with_constants, const_value, const_type)) { - if (traverseASTEquals(function->name, arguments[1], const_type, const_value, out)) + if (traverseASTEquals(function->name, arguments[1], const_type, const_value, out, parent)) maybe_useful = true; } } @@ -323,8 +324,50 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTIn( return false; } +static bool indexOfCanUseBloomFilter(const ASTPtr & parent) +{ + if (!parent) + return true; + + if (const auto * function = parent->as()) + { + if (function->name == "arrayElement") + { + return true; + } + else if (function->name == "equals" + || function->name == "greater" || function->name == "greaterOrEquals" + || function->name == "less" || function->name == "lessOrEquals") + { + if (function->arguments->children.size() != 2) + return false; + + if (const ASTLiteral * left = function->arguments->children[0]->as()) + { + if (function->name == "equals" && left->value.get() != 0) + return true; + else if (function->name == "less" && left->value.get() >= 0) + return true; + else if (function->name == "lessOrEquals" && left->value.get() > 0) + return true; + } + else if (const ASTLiteral * right = function->arguments->children[1]->as()) + { + if (function->name == "equals" && right->value.get() != 0) + return true; + else if (function->name == "greater" && right->value.get() >= 0) + return true; + else if (function->name == "greaterOrEquals" && right->value.get() > 0) + return true; + } + } + } + + return false; +} + bool MergeTreeIndexConditionBloomFilter::traverseASTEquals( - const String & function_name, const ASTPtr & key_ast, const DataTypePtr & value_type, const Field & value_field, RPNElement & out) + const String & function_name, const ASTPtr & key_ast, const DataTypePtr & value_type, const Field & value_field, RPNElement & out, const ASTPtr & parent) { if (header.has(key_ast->getColumnName())) { @@ -334,19 +377,21 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals( if (function_name == "has" || function_name == "indexOf") { - out.function = RPNElement::FUNCTION_HAS; - if (!array_type) - throw Exception("First argument for function has must be an array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception("First argument for function " + function_name + " must be an array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - const DataTypePtr actual_type = BloomFilter::getPrimitiveType(array_type->getNestedType()); - Field converted_field = convertFieldToType(value_field, *actual_type, value_type.get()); - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), converted_field))); + if (function_name == "has" || indexOfCanUseBloomFilter(parent)) + { + out.function = RPNElement::FUNCTION_HAS; + const DataTypePtr actual_type = BloomFilter::getPrimitiveType(array_type->getNestedType()); + Field converted_field = convertFieldToType(value_field, *actual_type, value_type.get()); + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), converted_field))); + } } else { if (array_type) - throw Exception("An array type of bloom_filter supports only has() function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception("An array type of bloom_filter supports only has() and indexOf() function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); out.function = function_name == "equals" ? RPNElement::FUNCTION_EQUALS : RPNElement::FUNCTION_NOT_EQUALS; const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type); @@ -374,7 +419,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals( const DataTypes & subtypes = value_tuple_data_type->getElements(); for (size_t index = 0; index < tuple.size(); ++index) - match_with_subtype |= traverseASTEquals(function_name, arguments[index], subtypes[index], tuple[index], out); + match_with_subtype |= traverseASTEquals(function_name, arguments[index], subtypes[index], tuple[index], out, key_ast); return match_with_subtype; } diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h index 40299cb7a43..34fb45c86a5 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h @@ -67,7 +67,7 @@ private: bool traverseAtomAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out); - bool traverseFunction(const ASTPtr & node, Block & block_with_constants, RPNElement & out); + bool traverseFunction(const ASTPtr & node, Block & block_with_constants, RPNElement & out, const ASTPtr & parent); bool traverseASTIn(const String & function_name, const ASTPtr & key_ast, const SetPtr & prepared_set, RPNElement & out); @@ -75,7 +75,7 @@ private: const String & function_name, const ASTPtr & key_ast, const DataTypePtr & type, const ColumnPtr & column, RPNElement & out); bool traverseASTEquals( - const String & function_name, const ASTPtr & key_ast, const DataTypePtr & value_type, const Field & value_field, RPNElement & out); + const String & function_name, const ASTPtr & key_ast, const DataTypePtr & value_type, const Field & value_field, RPNElement & out, const ASTPtr & parent); }; } diff --git a/tests/queries/0_stateless/00945_bloom_filter_index.reference b/tests/queries/0_stateless/00945_bloom_filter_index.reference index a00ae5f2d5b..184aafdd568 100755 --- a/tests/queries/0_stateless/00945_bloom_filter_index.reference +++ b/tests/queries/0_stateless/00945_bloom_filter_index.reference @@ -178,3 +178,39 @@ 5000 5000 5000 +2 +1 +2 +1 +1 +2 +1 +2 +1 +2 +2 +1 +1 +2 +2 +1 +2 +1 +2 +2 +1 +1 +2 +1 +2 +1 +1 +1 +2 +1 +2 +1 +1 +1 value1 +1 value2 +2 value3 diff --git a/tests/queries/0_stateless/00945_bloom_filter_index.sql b/tests/queries/0_stateless/00945_bloom_filter_index.sql old mode 100755 new mode 100644 index 083caba710b..82321a75c67 --- a/tests/queries/0_stateless/00945_bloom_filter_index.sql +++ b/tests/queries/0_stateless/00945_bloom_filter_index.sql @@ -302,3 +302,54 @@ CREATE TABLE bloom_filter_array_offsets_i (order_key int, i Array(int), INDEX id INSERT INTO bloom_filter_array_offsets_i SELECT number AS i, if(i%2, [99999], []) FROM system.numbers LIMIT 10000; SELECT count() FROM bloom_filter_array_offsets_i WHERE has(i, 99999); DROP TABLE IF EXISTS bloom_filter_array_offsets_i; + +DROP TABLE IF EXISTS test_bf_indexOf; +CREATE TABLE test_bf_indexOf ( `id` int, `ary` Array(LowCardinality(Nullable(String))), INDEX idx_ary ary TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 1; +INSERT INTO test_bf_indexOf VALUES (1, ['value1', 'value2']); +INSERT INTO test_bf_indexOf VALUES (2, ['value3']); + +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') = 0 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') = 1 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value2') = 0 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value2') = 2 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value3') = 0 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value3') = 1 ORDER BY id FORMAT TSV; + +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') != 0 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') != 1 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value2') != 0 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value2') != 2 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value3') != 0 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value3') != 1 ORDER BY id FORMAT TSV; + +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') = 2 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value3') = 2 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') = 1 OR indexOf(ary, 'value3') = 1 ORDER BY id FORMAT TSV; + +SELECT id FROM test_bf_indexOf WHERE not(indexOf(ary, 'value1')) ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE not(indexOf(ary, 'value1') == 0) ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE not(indexOf(ary, 'value1') == 1) ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE not(indexOf(ary, 'value1') == 2) ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') in (0) ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') in (1) ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') in (2) ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') not in (0) ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') not in (1) ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') not in (2) ORDER BY id FORMAT TSV; + +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') > 0 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE 0 < indexOf(ary, 'value1') ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') >= 0 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE 0 <= indexOf(ary, 'value1') ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') > 1 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE 1 < indexOf(ary, 'value1') ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') >= 1 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE 1 <= indexOf(ary, 'value1') ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE indexOf(ary, 'value1') >= 2 ORDER BY id FORMAT TSV; +SELECT id FROM test_bf_indexOf WHERE 2 <= indexOf(ary, 'value1') ORDER BY id FORMAT TSV; + +SELECT id, ary[indexOf(ary, 'value1')] FROM test_bf_indexOf WHERE ary[indexOf(ary, 'value1')] = 'value1' ORDER BY id FORMAT TSV; +SELECT id, ary[indexOf(ary, 'value2')] FROM test_bf_indexOf WHERE ary[indexOf(ary, 'value2')] = 'value2' ORDER BY id FORMAT TSV; +SELECT id, ary[indexOf(ary, 'value3')] FROM test_bf_indexOf WHERE ary[indexOf(ary, 'value3')] = 'value3' ORDER BY id FORMAT TSV; + +DROP TABLE IF EXISTS test_bf_indexOf; From 2045bc9f8c2f2e64d9669fd5c78a72e0fbacb89b Mon Sep 17 00:00:00 2001 From: "philip.han" Date: Sun, 20 Sep 2020 19:06:46 +0900 Subject: [PATCH 010/314] Fix performance_test for bloom_filter --- tests/performance/bloom_filter.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/performance/bloom_filter.xml b/tests/performance/bloom_filter.xml index d1881cde56e..00bdd594738 100644 --- a/tests/performance/bloom_filter.xml +++ b/tests/performance/bloom_filter.xml @@ -1,12 +1,13 @@ DROP TABLE IF EXISTS test_bf CREATE TABLE test_bf (`id` int, `ary` Array(String), INDEX idx_ary ary TYPE bloom_filter(0.01) GRANULARITY 8192) ENGINE = MergeTree() ORDER BY id - CREATE TABLE test_bf_indexOf (`id` int, `ary` Array(String), INDEX idx_ary ary TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY id + CREATE TABLE test_bf_indexOf (`id` int, `ary` Array(LowCardinality(Nullable(String))), INDEX idx_ary ary TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY id SYSTEM STOP MERGES INSERT INTO test_bf SELECT number AS id, [CAST(id, 'String'), CAST(id + 1, 'String'), CAST(id + 2, 'String')] FROM numbers(1000000) INSERT INTO test_bf_indexOf SELECT number AS id, [CAST(id, 'String'), CAST(id + 1, 'String'), CAST(id + 2, 'String')] FROM numbers(1000000) + SELECT count() FROM test_bf_indexOf WHERE indexOf(ary, '1') = 2 SELECT count() FROM test_bf_indexOf WHERE indexOf(ary, '1') > 0 SELECT count() FROM test_bf_indexOf WHERE ary[indexOf(ary, '1')] = '1' From 7197a0c803f8ee65d5241b76fe09533e31b3f684 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Thu, 1 Oct 2020 11:38:50 +0100 Subject: [PATCH 011/314] Possibly fix the issue with quorum --- src/Storages/StorageReplicatedMergeTree.cpp | 72 ++++++++++++++++++++- src/Storages/StorageReplicatedMergeTree.h | 2 + 2 files changed, 72 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e8007440f2f..23cca6a818e 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3107,6 +3107,60 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id) } } +void StorageReplicatedMergeTree::updateLastPartNodeIfMatches(const String & partition_id, const String & old_part_name, const String & new_part_name) +{ + auto zookeeper = getZooKeeper(); + + const String quorum_last_part_path = zookeeper_path + "/quorum/last_part"; + + while (true) + { + Coordination::Stat added_parts_stat; + String old_added_parts = zookeeper->get(quorum_last_part_path, &added_parts_stat); + + ReplicatedMergeTreeQuorumAddedParts parts_with_quorum(format_version); + + if (!old_added_parts.empty()) + parts_with_quorum.fromString(old_added_parts); + + if (!parts_with_quorum.added_parts.count(partition_id)) + { + /// There is no information about partition at all. + break; + } + + /// Part for which last quorum was reached in partition_id. + auto quorum_part_info = MergeTreePartInfo::fromPartName(parts_with_quorum.added_parts.at(partition_id), format_version); + auto old_part_info = MergeTreePartInfo::fromPartName(old_part_name, format_version); + + /// Update last part for which quorum was reached. + if (old_part_info.contains(quorum_part_info)) + parts_with_quorum.added_parts.emplace(partition_id, new_part_name); + + /// Serialize and try update. + String new_added_parts = parts_with_quorum.toString(); + + auto code = zookeeper->trySet(quorum_last_part_path, new_added_parts, added_parts_stat.version); + + if (code == Coordination::Error::ZOK) + { + break; + } + else if (code == Coordination::Error::ZNONODE) + { + /// Node is deleted. It is impossible, but it is Ok. + break; + } + else if (code == Coordination::Error::ZBADVERSION) + { + /// Node was updated meanwhile. We must re-read it and repeat all the actions. + continue; + } + else + throw Coordination::Exception(code, quorum_last_part_path); + } +} + bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const StorageMetadataPtr & metadata_snapshot, const String & source_replica_path, bool to_detached, size_t quorum) @@ -4055,9 +4109,23 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool de } } + bool drop_entire_partition = !drop_part; + /// Cleaning possibly stored information about parts from /quorum/last_part node in ZooKeeper. - /// TODO(nv) how is this related to dropPart? Is it? - if (!drop_part) + if (drop_part) + { + auto part_info = MergeTreePartInfo::fromPartName(partition->as().value.safeGet(), format_version); + auto data_parts_vec = getDataPartsVectorInPartition(DataPartState::Committed, part_info.partition_id); + std::sort(data_parts_vec.begin(), data_parts_vec.end(), LessDataPart()); + + auto prev_part = std::upper_bound(data_parts_vec.begin(), data_parts_vec.end(), part_info, LessDataPart()); + if (prev_part != data_parts_vec.end()) + updateLastPartNodeIfMatches(part_info.partition_id, part_info.getPartName(), (*prev_part)->info.getPartName()); + else if (data_parts_vec.empty()) + drop_entire_partition = true; + } + + if (drop_entire_partition) { String partition_id = getPartitionIDFromQuery(partition, query_context); cleanLastPartNode(partition_id); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 3ce946a54c0..4fb57f6a10b 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -490,6 +490,8 @@ private: /// Deletes info from quorum/last_part node for particular partition_id. void cleanLastPartNode(const String & partition_id); + void updateLastPartNodeIfMatches(const String & partition_id, const String & old_part_name, const String & new_part_name); + /// Creates new block number if block with such block_id does not exist std::optional allocateBlockNumber( const String & partition_id, zkutil::ZooKeeperPtr & zookeeper, From 587cde853ecd45196394915705aa6872af1e3dd7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 10 Sep 2020 22:55:36 +0300 Subject: [PATCH 012/314] Avoid skipping unused shards twice (for query processing stage and read itself) --- src/DataStreams/RemoteQueryExecutor.cpp | 6 ++- .../ClusterProxy/executeQuery.cpp | 10 ++-- src/Interpreters/ClusterProxy/executeQuery.h | 4 +- src/Interpreters/InterpreterSelectQuery.cpp | 5 +- src/Storages/IStorage.h | 9 ++-- src/Storages/LiveView/StorageBlocks.h | 2 +- src/Storages/SelectQueryInfo.h | 6 +++ src/Storages/StorageBuffer.cpp | 4 +- src/Storages/StorageBuffer.h | 2 +- src/Storages/StorageDistributed.cpp | 52 ++++++++----------- src/Storages/StorageDistributed.h | 2 +- src/Storages/StorageMaterializedView.cpp | 4 +- src/Storages/StorageMaterializedView.h | 2 +- src/Storages/StorageMerge.cpp | 6 +-- src/Storages/StorageMerge.h | 2 +- src/Storages/tests/gtest_storage_log.cpp | 5 +- 16 files changed, 63 insertions(+), 58 deletions(-) diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index 1b1bf2af8ef..9d036f96c42 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -314,6 +315,8 @@ void RemoteQueryExecutor::sendScalars() void RemoteQueryExecutor::sendExternalTables() { + SelectQueryInfo query_info; + size_t count = multiplexed_connections->size(); { @@ -328,7 +331,8 @@ void RemoteQueryExecutor::sendExternalTables() { StoragePtr cur = table.second; auto metadata_snapshot = cur->getInMemoryMetadataPtr(); - QueryProcessingStage::Enum read_from_table_stage = cur->getQueryProcessingStage(context); + QueryProcessingStage::Enum read_from_table_stage = cur->getQueryProcessingStage( + context, QueryProcessingStage::Complete, query_info); Pipe pipe = cur->read( metadata_snapshot->getColumns().getNamesOfPhysical(), diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 1ebd3009ff7..74b9cd64ffc 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -4,9 +4,10 @@ #include #include #include -#include #include +#include #include +#include namespace DB @@ -75,12 +76,13 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin } Pipe executeQuery( - IStreamFactory & stream_factory, const ClusterPtr & cluster, Poco::Logger * log, - const ASTPtr & query_ast, const Context & context, const Settings & settings, const SelectQueryInfo & query_info) + IStreamFactory & stream_factory, Poco::Logger * log, + const ASTPtr & query_ast, const Context & context, const SelectQueryInfo & query_info) { assert(log); Pipes res; + const Settings & settings = context.getSettingsRef(); const std::string query = queryToString(query_ast); @@ -103,7 +105,7 @@ Pipe executeQuery( else throttler = user_level_throttler; - for (const auto & shard_info : cluster->getShardsInfo()) + for (const auto & shard_info : query_info.cluster->getShardsInfo()) stream_factory.createForShard(shard_info, query, query_ast, new_context, throttler, query_info, res); return Pipe::unitePipes(std::move(res)); diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index f0d9539770d..389b0bd7115 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -1,7 +1,6 @@ #pragma once #include -#include namespace DB { @@ -26,8 +25,7 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin /// `stream_factory` object encapsulates the logic of creating streams for a different type of query /// (currently SELECT, DESCRIBE). Pipe executeQuery( - IStreamFactory & stream_factory, const ClusterPtr & cluster, Poco::Logger * log, - const ASTPtr & query_ast, const Context & context, const Settings & settings, const SelectQueryInfo & query_info); + IStreamFactory & stream_factory, Poco::Logger * log, const ASTPtr & query_ast, const Context & context, const SelectQueryInfo & query_info); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 556070d0360..119dda6a3d4 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -489,8 +489,10 @@ BlockIO InterpreterSelectQuery::execute() Block InterpreterSelectQuery::getSampleBlockImpl() { + query_info.query = query_ptr; + if (storage && !options.only_analyze) - from_stage = storage->getQueryProcessingStage(*context, options.to_stage, query_ptr); + from_stage = storage->getQueryProcessingStage(*context, options.to_stage, query_info); /// Do I need to perform the first part of the pipeline - running on remote servers during distributed processing. bool first_stage = from_stage < QueryProcessingStage::WithMergeableState @@ -1415,7 +1417,6 @@ void InterpreterSelectQuery::executeFetchColumns( if (max_streams > 1 && !is_remote) max_streams *= settings.max_streams_to_max_threads_ratio; - query_info.query = query_ptr; query_info.syntax_analyzer_result = syntax_analyzer_result; query_info.sets = query_analyzer->getPreparedSets(); query_info.prewhere_info = prewhere_info; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 4a2e70aa84b..e03f6f0bf17 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include @@ -55,6 +54,7 @@ using StoragePolicyPtr = std::shared_ptr; struct StreamLocalLimits; class EnabledQuota; +struct SelectQueryInfo; struct ColumnSize { @@ -212,15 +212,12 @@ public: * * SelectQueryInfo is required since the stage can depends on the query * (see Distributed() engine and optimize_skip_unused_shards). + * And to store optimized cluster (after optimize_skip_unused_shards). * * QueryProcessingStage::Enum required for Distributed over Distributed, * since it cannot return Complete for intermediate queries never. */ - QueryProcessingStage::Enum getQueryProcessingStage(const Context & context) const - { - return getQueryProcessingStage(context, QueryProcessingStage::Complete, {}); - } - virtual QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const ASTPtr &) const + virtual QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const SelectQueryInfo &) const { return QueryProcessingStage::FetchColumns; } diff --git a/src/Storages/LiveView/StorageBlocks.h b/src/Storages/LiveView/StorageBlocks.h index 37861b55568..6ca86132b28 100644 --- a/src/Storages/LiveView/StorageBlocks.h +++ b/src/Storages/LiveView/StorageBlocks.h @@ -33,7 +33,7 @@ public: bool supportsSampling() const override { return true; } bool supportsFinal() const override { return true; } - QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const ASTPtr &) const override { return to_stage; } + QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const SelectQueryInfo &) const override { return to_stage; } Pipe read( const Names & /*column_names*/, diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 9a5d0cc6338..f5069169b55 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -63,6 +63,8 @@ using TreeRewriterResultPtr = std::shared_ptr; class ReadInOrderOptimizer; using ReadInOrderOptimizerPtr = std::shared_ptr; +class Cluster; +using ClusterPtr = std::shared_ptr; /** Query along with some additional data, * that can be used during query processing @@ -73,6 +75,10 @@ struct SelectQueryInfo ASTPtr query; ASTPtr view_query; /// Optimized VIEW query + /// For optimize_skip_unused_shards + /// We can modify it in getQueryProcessingStage() + mutable ClusterPtr cluster; + TreeRewriterResultPtr syntax_analyzer_result; PrewhereInfoPtr prewhere_info; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 4b038692b77..91d5fbac3c0 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -130,7 +130,7 @@ private: }; -QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, const ASTPtr & query_ptr) const +QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, const SelectQueryInfo & query_info) const { if (destination_id) { @@ -139,7 +139,7 @@ QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(const Context if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); - return destination->getQueryProcessingStage(context, to_stage, query_ptr); + return destination->getQueryProcessingStage(context, to_stage, query_info); } return QueryProcessingStage::FetchColumns; diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index b18b574ec6c..6f81dc60346 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -54,7 +54,7 @@ public: std::string getName() const override { return "Buffer"; } - QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const ASTPtr &) const override; + QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const SelectQueryInfo &) const override; Pipe read( const Names & column_names, diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b06434b6317..c961ea14f41 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -446,11 +446,31 @@ StoragePtr StorageDistributed::createWithOwnCluster( return res; } -QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Context &context, QueryProcessingStage::Enum to_stage, const ASTPtr & query_ptr) const +QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, const SelectQueryInfo & query_info) const { const auto & settings = context.getSettingsRef(); auto metadata_snapshot = getInMemoryMetadataPtr(); + ClusterPtr cluster = getCluster(); + query_info.cluster = cluster; + + /// Always calculate optimized cluster here, to avoid conditions during read() + /// (Anyway it will be calculated in the read()) + if (settings.optimize_skip_unused_shards) + { + ClusterPtr optimized_cluster = getOptimizedCluster(context, metadata_snapshot, query_info.query); + if (optimized_cluster) + { + LOG_DEBUG(log, "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}", makeFormattedListOfShards(optimized_cluster)); + cluster = optimized_cluster; + query_info.cluster = cluster; + } + else + { + LOG_DEBUG(log, "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster{}", has_sharding_key ? "" : " (no sharding key)"); + } + } + if (settings.distributed_group_by_no_merge) { if (settings.distributed_group_by_no_merge == DISTRIBUTED_GROUP_BY_NO_MERGE_AFTER_AGGREGATION) @@ -464,14 +484,6 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Con if (to_stage == QueryProcessingStage::WithMergeableState) return QueryProcessingStage::WithMergeableState; - ClusterPtr cluster = getCluster(); - if (settings.optimize_skip_unused_shards) - { - ClusterPtr optimized_cluster = getOptimizedCluster(context, metadata_snapshot, query_ptr); - if (optimized_cluster) - cluster = optimized_cluster; - } - /// If there is only one node, the query can be fully processed by the /// shard, initiator will work as a proxy only. if (getClusterQueriedNodes(settings, cluster) == 1) @@ -483,7 +495,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Con sharding_key_is_deterministic) { Block sharding_key_block = sharding_key_expr->getSampleBlock(); - auto stage = getOptimizedQueryProcessingStage(query_ptr, settings.extremes, sharding_key_block); + auto stage = getOptimizedQueryProcessingStage(query_info.query, settings.extremes, sharding_key_block); if (stage) { LOG_DEBUG(log, "Force processing stage to {}", QueryProcessingStage::toString(*stage)); @@ -503,23 +515,6 @@ Pipe StorageDistributed::read( const size_t /*max_block_size*/, const unsigned /*num_streams*/) { - const auto & settings = context.getSettingsRef(); - - ClusterPtr cluster = getCluster(); - if (settings.optimize_skip_unused_shards) - { - ClusterPtr optimized_cluster = getOptimizedCluster(context, metadata_snapshot, query_info.query); - if (optimized_cluster) - { - LOG_DEBUG(log, "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}", makeFormattedListOfShards(optimized_cluster)); - cluster = optimized_cluster; - } - else - { - LOG_DEBUG(log, "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster{}", has_sharding_key ? "" : " (no sharding key)"); - } - } - const auto & modified_query_ast = rewriteSelectQuery( query_info.query, remote_database, remote_table, remote_table_function_ptr); @@ -538,8 +533,7 @@ Pipe StorageDistributed::read( : ClusterProxy::SelectStreamFactory( header, processed_stage, StorageID{remote_database, remote_table}, scalars, has_virtual_shard_num_column, context.getExternalTables()); - return ClusterProxy::executeQuery(select_stream_factory, cluster, log, - modified_query_ast, context, context.getSettingsRef(), query_info); + return ClusterProxy::executeQuery(select_stream_factory, log, modified_query_ast, context, query_info); } diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 7e4e9f56ab4..382bca31e6a 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -66,7 +66,7 @@ public: bool isRemote() const override { return true; } - QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum to_stage, const ASTPtr &) const override; + QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const SelectQueryInfo &) const override; Pipe read( const Names & column_names, diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index a2e3fae0951..47de24768ed 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -101,9 +101,9 @@ StorageMaterializedView::StorageMaterializedView( DatabaseCatalog::instance().addDependency(select.select_table_id, getStorageID()); } -QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, const ASTPtr & query_ptr) const +QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, const SelectQueryInfo & query_info) const { - return getTargetTable()->getQueryProcessingStage(context, to_stage, query_ptr); + return getTargetTable()->getQueryProcessingStage(context, to_stage, query_info); } Pipe StorageMaterializedView::read( diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 1ee4246c7f1..ac2c2604ae1 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -64,7 +64,7 @@ public: void checkTableCanBeDropped() const override; void checkPartitionCanBeDropped(const ASTPtr & partition) override; - QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const ASTPtr &) const override; + QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const SelectQueryInfo &) const override; StoragePtr getTargetTable() const; StoragePtr tryGetTargetTable() const; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index bade0810320..1fde265e447 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -149,7 +149,7 @@ bool StorageMerge::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, cons } -QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, const ASTPtr & query_ptr) const +QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, const SelectQueryInfo & query_info) const { auto stage_in_source_tables = QueryProcessingStage::FetchColumns; @@ -163,7 +163,7 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context & if (table && table.get() != this) { ++selected_table_size; - stage_in_source_tables = std::max(stage_in_source_tables, table->getQueryProcessingStage(context, to_stage, query_ptr)); + stage_in_source_tables = std::max(stage_in_source_tables, table->getQueryProcessingStage(context, to_stage, query_info)); } iterator->next(); @@ -308,7 +308,7 @@ Pipe StorageMerge::createSources( return pipe; } - auto storage_stage = storage->getQueryProcessingStage(*modified_context, QueryProcessingStage::Complete, modified_query_info.query); + auto storage_stage = storage->getQueryProcessingStage(*modified_context, QueryProcessingStage::Complete, modified_query_info); if (processed_stage <= storage_stage) { /// If there are only virtual columns in query, you must request at least one other column. diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 5e2d56d18c0..5929e418755 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -27,7 +27,7 @@ public: bool supportsFinal() const override { return true; } bool supportsIndexForIn() const override { return true; } - QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const ASTPtr &) const override; + QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const SelectQueryInfo &) const override; Pipe read( const Names & column_names, diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index 8de14b53471..1eab072fb4b 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -114,7 +115,9 @@ std::string readData(DB::StoragePtr & table, const DB::Context & context) Names column_names; column_names.push_back("a"); - QueryProcessingStage::Enum stage = table->getQueryProcessingStage(context); + SelectQueryInfo query_info; + QueryProcessingStage::Enum stage = table->getQueryProcessingStage( + context, QueryProcessingStage::Complete, query_info); QueryPipeline pipeline; pipeline.init(table->read(column_names, metadata_snapshot, {}, context, stage, 8192, 1)); From b838214a35a84117333e8e5a2764e28e4ec5b952 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 20 Sep 2020 20:52:17 +0300 Subject: [PATCH 013/314] Pass non-const SelectQueryInfo (and drop mutable qualifiers) --- src/DataStreams/RemoteQueryExecutor.cpp | 2 +- src/Processors/QueryPlan/ReadFromStorageStep.cpp | 2 +- src/Processors/QueryPlan/ReadFromStorageStep.h | 4 ++-- src/Storages/IStorage.cpp | 4 ++-- src/Storages/IStorage.h | 6 +++--- src/Storages/Kafka/StorageKafka.cpp | 2 +- src/Storages/Kafka/StorageKafka.h | 2 +- src/Storages/LiveView/StorageBlocks.h | 4 ++-- src/Storages/LiveView/StorageLiveView.cpp | 2 +- src/Storages/LiveView/StorageLiveView.h | 2 +- src/Storages/MergeTree/StorageFromMergeTreeDataPart.h | 2 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 2 +- src/Storages/RabbitMQ/StorageRabbitMQ.h | 2 +- src/Storages/SelectQueryInfo.h | 10 +++++----- src/Storages/StorageBuffer.cpp | 4 ++-- src/Storages/StorageBuffer.h | 4 ++-- src/Storages/StorageDictionary.cpp | 2 +- src/Storages/StorageDictionary.h | 2 +- src/Storages/StorageDistributed.cpp | 4 ++-- src/Storages/StorageDistributed.h | 4 ++-- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageFile.h | 2 +- src/Storages/StorageGenerateRandom.cpp | 2 +- src/Storages/StorageGenerateRandom.h | 2 +- src/Storages/StorageHDFS.cpp | 2 +- src/Storages/StorageHDFS.h | 2 +- src/Storages/StorageInput.cpp | 2 +- src/Storages/StorageInput.h | 2 +- src/Storages/StorageJoin.cpp | 2 +- src/Storages/StorageJoin.h | 2 +- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageLog.h | 2 +- src/Storages/StorageMaterializeMySQL.cpp | 2 +- src/Storages/StorageMaterializeMySQL.h | 2 +- src/Storages/StorageMaterializedView.cpp | 4 ++-- src/Storages/StorageMaterializedView.h | 4 ++-- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMemory.h | 2 +- src/Storages/StorageMerge.cpp | 8 ++++---- src/Storages/StorageMerge.h | 8 ++++---- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageMongoDB.cpp | 2 +- src/Storages/StorageMongoDB.h | 2 +- src/Storages/StorageMySQL.cpp | 2 +- src/Storages/StorageMySQL.h | 2 +- src/Storages/StorageNull.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.h | 2 +- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageS3.h | 2 +- src/Storages/StorageStripeLog.cpp | 2 +- src/Storages/StorageStripeLog.h | 2 +- src/Storages/StorageTinyLog.cpp | 2 +- src/Storages/StorageTinyLog.h | 2 +- src/Storages/StorageURL.cpp | 2 +- src/Storages/StorageURL.h | 2 +- src/Storages/StorageValues.cpp | 2 +- src/Storages/StorageValues.h | 2 +- src/Storages/StorageView.cpp | 2 +- src/Storages/StorageView.h | 2 +- src/Storages/StorageXDBC.cpp | 2 +- src/Storages/StorageXDBC.h | 2 +- src/Storages/System/IStorageSystemOneBlock.h | 2 +- src/Storages/System/StorageSystemColumns.cpp | 2 +- src/Storages/System/StorageSystemColumns.h | 2 +- src/Storages/System/StorageSystemDetachedParts.cpp | 2 +- src/Storages/System/StorageSystemDetachedParts.h | 2 +- src/Storages/System/StorageSystemDisks.cpp | 2 +- src/Storages/System/StorageSystemDisks.h | 2 +- src/Storages/System/StorageSystemNumbers.cpp | 2 +- src/Storages/System/StorageSystemNumbers.h | 2 +- src/Storages/System/StorageSystemOne.cpp | 2 +- src/Storages/System/StorageSystemOne.h | 2 +- src/Storages/System/StorageSystemPartsBase.cpp | 2 +- src/Storages/System/StorageSystemPartsBase.h | 2 +- src/Storages/System/StorageSystemReplicas.cpp | 2 +- src/Storages/System/StorageSystemReplicas.h | 2 +- src/Storages/System/StorageSystemStoragePolicies.cpp | 2 +- src/Storages/System/StorageSystemStoragePolicies.h | 2 +- src/Storages/System/StorageSystemTables.cpp | 2 +- src/Storages/System/StorageSystemTables.h | 2 +- src/Storages/System/StorageSystemZeros.cpp | 2 +- src/Storages/System/StorageSystemZeros.h | 2 +- src/Storages/tests/gtest_storage_log.cpp | 2 +- 85 files changed, 106 insertions(+), 106 deletions(-) diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index 9d036f96c42..38486aa6368 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -336,7 +336,7 @@ void RemoteQueryExecutor::sendExternalTables() Pipe pipe = cur->read( metadata_snapshot->getColumns().getNamesOfPhysical(), - metadata_snapshot, {}, context, + metadata_snapshot, query_info, context, read_from_table_stage, DEFAULT_BLOCK_SIZE, 1); auto data = std::make_unique(); diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.cpp b/src/Processors/QueryPlan/ReadFromStorageStep.cpp index b085c177ad4..2a656ddac00 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromStorageStep.cpp @@ -19,7 +19,7 @@ ReadFromStorageStep::ReadFromStorageStep( std::shared_ptr quota_, StoragePtr storage_, const Names & required_columns_, - const SelectQueryInfo & query_info_, + SelectQueryInfo & query_info_, std::shared_ptr context_, QueryProcessingStage::Enum processing_stage_, size_t max_block_size_, diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.h b/src/Processors/QueryPlan/ReadFromStorageStep.h index 98cde63a863..db62ff54a08 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.h +++ b/src/Processors/QueryPlan/ReadFromStorageStep.h @@ -30,7 +30,7 @@ public: std::shared_ptr quota, StoragePtr storage, const Names & required_columns, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, std::shared_ptr context, QueryProcessingStage::Enum processing_stage, size_t max_block_size, @@ -53,7 +53,7 @@ private: StoragePtr storage; const Names & required_columns; - const SelectQueryInfo & query_info; + SelectQueryInfo & query_info; std::shared_ptr context; QueryProcessingStage::Enum processing_stage; size_t max_block_size; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 50b36ced19c..d2305f6416e 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -83,7 +83,7 @@ TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, cons Pipe IStorage::read( const Names & /*column_names*/, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, @@ -100,7 +100,7 @@ void IStorage::read( SizeLimits & leaf_limits, std::shared_ptr quota, const Names & column_names, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, std::shared_ptr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index e03f6f0bf17..5227c12ac85 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -217,7 +217,7 @@ public: * QueryProcessingStage::Enum required for Distributed over Distributed, * since it cannot return Complete for intermediate queries never. */ - virtual QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const SelectQueryInfo &) const + virtual QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, SelectQueryInfo &) const { return QueryProcessingStage::FetchColumns; } @@ -275,7 +275,7 @@ public: virtual Pipe read( const Names & /*column_names*/, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, @@ -291,7 +291,7 @@ public: SizeLimits & leaf_limits, std::shared_ptr quota, const Names & column_names, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, std::shared_ptr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 9ba5ad7a65b..392d8cc8886 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -206,7 +206,7 @@ String StorageKafka::getDefaultClientId(const StorageID & table_id_) Pipe StorageKafka::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & /* query_info */, + SelectQueryInfo & /* query_info */, const Context & context, QueryProcessingStage::Enum /* processed_stage */, size_t /* max_block_size */, diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 272e419bebe..44d8baca299 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -41,7 +41,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/LiveView/StorageBlocks.h b/src/Storages/LiveView/StorageBlocks.h index 6ca86132b28..4ad0ffb93ca 100644 --- a/src/Storages/LiveView/StorageBlocks.h +++ b/src/Storages/LiveView/StorageBlocks.h @@ -33,12 +33,12 @@ public: bool supportsSampling() const override { return true; } bool supportsFinal() const override { return true; } - QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const SelectQueryInfo &) const override { return to_stage; } + QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, SelectQueryInfo &) const override { return to_stage; } Pipe read( const Names & /*column_names*/, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 7095357a161..4e67b15e6d7 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -424,7 +424,7 @@ void StorageLiveView::refresh() Pipe StorageLiveView::read( const Names & /*column_names*/, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index fe4be6ee08e..f6ec129fe76 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -129,7 +129,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index c13f540ad34..aa24ddcf33c 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -21,7 +21,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 78aad9e6a2d..bdf110fc7c3 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -479,7 +479,7 @@ void StorageRabbitMQ::unbindExchange() Pipe StorageRabbitMQ::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & /* query_info */, + SelectQueryInfo & /* query_info */, const Context & context, QueryProcessingStage::Enum /* processed_stage */, size_t /* max_block_size */, diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 8d9a20f9e34..64243035fd3 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -38,7 +38,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index f5069169b55..029f033dbd7 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -75,17 +75,17 @@ struct SelectQueryInfo ASTPtr query; ASTPtr view_query; /// Optimized VIEW query - /// For optimize_skip_unused_shards - /// We can modify it in getQueryProcessingStage() - mutable ClusterPtr cluster; + /// For optimize_skip_unused_shards. + /// Can be modified in getQueryProcessingStage() + ClusterPtr cluster; TreeRewriterResultPtr syntax_analyzer_result; PrewhereInfoPtr prewhere_info; ReadInOrderOptimizerPtr order_optimizer; - /// We can modify it while reading from storage - mutable InputOrderInfoPtr input_order_info; + /// Can be modified while reading from storage + InputOrderInfoPtr input_order_info; /// Prepared sets are used for indices by storage engine. /// Example: x IN (1, 2, 3) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 91d5fbac3c0..846f47b11c6 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -130,7 +130,7 @@ private: }; -QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, const SelectQueryInfo & query_info) const +QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, SelectQueryInfo & query_info) const { if (destination_id) { @@ -149,7 +149,7 @@ QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(const Context Pipe StorageBuffer::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 6f81dc60346..455560e22da 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -54,12 +54,12 @@ public: std::string getName() const override { return "Buffer"; } - QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const SelectQueryInfo &) const override; + QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, SelectQueryInfo &) const override; Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index e859baa702e..2d84dde14ba 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -132,7 +132,7 @@ void StorageDictionary::checkTableCanBeDropped() const Pipe StorageDictionary::read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, const size_t max_block_size, diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index 5c7beb88d88..576cc2de064 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -19,7 +19,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index c961ea14f41..9ce3672f9a7 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -446,7 +446,7 @@ StoragePtr StorageDistributed::createWithOwnCluster( return res; } -QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, const SelectQueryInfo & query_info) const +QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, SelectQueryInfo & query_info) const { const auto & settings = context.getSettingsRef(); auto metadata_snapshot = getInMemoryMetadataPtr(); @@ -509,7 +509,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Con Pipe StorageDistributed::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, const size_t /*max_block_size*/, diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 382bca31e6a..a9d9a868166 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -66,12 +66,12 @@ public: bool isRemote() const override { return true; } - QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const SelectQueryInfo &) const override; + QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, SelectQueryInfo &) const override; Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 8f7e1563a62..10a38ed1068 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -381,7 +381,7 @@ private: Pipe StorageFile::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index ea70dcd5311..2e5ad988eab 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -27,7 +27,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 62d8259f705..f1e822be2d1 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -441,7 +441,7 @@ void registerStorageGenerateRandom(StorageFactory & factory) Pipe StorageGenerateRandom::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, diff --git a/src/Storages/StorageGenerateRandom.h b/src/Storages/StorageGenerateRandom.h index e0f037f9a08..965c5b3a9d3 100644 --- a/src/Storages/StorageGenerateRandom.h +++ b/src/Storages/StorageGenerateRandom.h @@ -18,7 +18,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageHDFS.cpp b/src/Storages/StorageHDFS.cpp index 65b1eed7793..7df7266445b 100644 --- a/src/Storages/StorageHDFS.cpp +++ b/src/Storages/StorageHDFS.cpp @@ -265,7 +265,7 @@ Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, c Pipe StorageHDFS::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & context_, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, diff --git a/src/Storages/StorageHDFS.h b/src/Storages/StorageHDFS.h index fdeaf4ae1b3..4172bce1cd1 100644 --- a/src/Storages/StorageHDFS.h +++ b/src/Storages/StorageHDFS.h @@ -22,7 +22,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageInput.cpp b/src/Storages/StorageInput.cpp index 5e525210548..1f881bccf07 100644 --- a/src/Storages/StorageInput.cpp +++ b/src/Storages/StorageInput.cpp @@ -61,7 +61,7 @@ void StorageInput::setInputStream(BlockInputStreamPtr input_stream_) Pipe StorageInput::read( const Names & /*column_names*/, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, diff --git a/src/Storages/StorageInput.h b/src/Storages/StorageInput.h index c19b19e4703..3cb64993d45 100644 --- a/src/Storages/StorageInput.h +++ b/src/Storages/StorageInput.h @@ -20,7 +20,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 33c67229cfa..d82e8ba959d 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -448,7 +448,7 @@ private: Pipe StorageJoin::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 95037c4d33a..d66820909aa 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -40,7 +40,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 944dc0e5804..86cc6afe33f 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -613,7 +613,7 @@ const StorageLog::Marks & StorageLog::getMarksWithRealRowCount(const StorageMeta Pipe StorageLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 49fc9a576c5..f28a3d67edb 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -27,7 +27,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageMaterializeMySQL.cpp b/src/Storages/StorageMaterializeMySQL.cpp index 7d908736bdc..32f110b8bef 100644 --- a/src/Storages/StorageMaterializeMySQL.cpp +++ b/src/Storages/StorageMaterializeMySQL.cpp @@ -34,7 +34,7 @@ StorageMaterializeMySQL::StorageMaterializeMySQL(const StoragePtr & nested_stora Pipe StorageMaterializeMySQL::read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageMaterializeMySQL.h b/src/Storages/StorageMaterializeMySQL.h index 4278ce64bd7..d97347ae789 100644 --- a/src/Storages/StorageMaterializeMySQL.h +++ b/src/Storages/StorageMaterializeMySQL.h @@ -22,7 +22,7 @@ public: StorageMaterializeMySQL(const StoragePtr & nested_storage_, const DatabaseMaterializeMySQL * database_); Pipe read( - const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, + const Names & column_names, const StorageMetadataPtr & metadata_snapshot, SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; NamesAndTypesList getVirtuals() const override; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 47de24768ed..db9c290cb66 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -101,7 +101,7 @@ StorageMaterializedView::StorageMaterializedView( DatabaseCatalog::instance().addDependency(select.select_table_id, getStorageID()); } -QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, const SelectQueryInfo & query_info) const +QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, SelectQueryInfo & query_info) const { return getTargetTable()->getQueryProcessingStage(context, to_stage, query_info); } @@ -109,7 +109,7 @@ QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(cons Pipe StorageMaterializedView::read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, const size_t max_block_size, diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index ac2c2604ae1..d4c218f67ba 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -64,7 +64,7 @@ public: void checkTableCanBeDropped() const override; void checkPartitionCanBeDropped(const ASTPtr & partition) override; - QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const SelectQueryInfo &) const override; + QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, SelectQueryInfo &) const override; StoragePtr getTargetTable() const; StoragePtr tryGetTargetTable() const; @@ -74,7 +74,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 25e232dc4ad..78bf06e5350 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -136,7 +136,7 @@ StorageMemory::StorageMemory(const StorageID & table_id_, ColumnsDescription col Pipe StorageMemory::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index e67e3015028..40d31ae2dd3 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -31,7 +31,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 1fde265e447..f4a057023f6 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -149,7 +149,7 @@ bool StorageMerge::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, cons } -QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, const SelectQueryInfo & query_info) const +QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, SelectQueryInfo & query_info) const { auto stage_in_source_tables = QueryProcessingStage::FetchColumns; @@ -176,7 +176,7 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context & Pipe StorageMerge::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, const size_t max_block_size, @@ -274,7 +274,7 @@ Pipe StorageMerge::read( Pipe StorageMerge::createSources( const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage, const UInt64 max_block_size, const Block & header, @@ -459,7 +459,7 @@ void StorageMerge::alter( Block StorageMerge::getQueryHeader( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage) { diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 5929e418755..5b4ea89e475 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -27,12 +27,12 @@ public: bool supportsFinal() const override { return true; } bool supportsIndexForIn() const override { return true; } - QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, const SelectQueryInfo &) const override; + QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, SelectQueryInfo &) const override; Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, @@ -78,13 +78,13 @@ protected: Block getQueryHeader( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage); Pipe createSources( const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage, const UInt64 max_block_size, const Block & header, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 55fb42b550e..698be543743 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -174,7 +174,7 @@ StorageMergeTree::~StorageMergeTree() Pipe StorageMergeTree::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, const size_t max_block_size, diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 5662f9e0088..318db847f0c 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -40,7 +40,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 210dc09ba86..b9ac2443472 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -56,7 +56,7 @@ StorageMongoDB::StorageMongoDB( Pipe StorageMongoDB::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, diff --git a/src/Storages/StorageMongoDB.h b/src/Storages/StorageMongoDB.h index a8bd2f4d160..cf2c5d8c278 100644 --- a/src/Storages/StorageMongoDB.h +++ b/src/Storages/StorageMongoDB.h @@ -37,7 +37,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index afbca0d9430..e5c59a794e1 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -67,7 +67,7 @@ StorageMySQL::StorageMySQL( Pipe StorageMySQL::read( const Names & column_names_, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info_, + SelectQueryInfo & query_info_, const Context & context_, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size_, diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index a7f98c4379b..acab8f9290e 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -40,7 +40,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index b5387c6b924..cc1d53a42e5 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -25,7 +25,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo &, + SelectQueryInfo &, const Context & /*context*/, QueryProcessingStage::Enum /*processing_stage*/, size_t, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 9613bd5111d..f40eb378d05 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3517,7 +3517,7 @@ ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock StorageReplicatedMerg Pipe StorageReplicatedMergeTree::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, const size_t max_block_size, diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index d851082d5c2..6635cb1b603 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -90,7 +90,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index ceca1405857..3c5310c1064 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -287,7 +287,7 @@ Strings listFilesWithRegexpMatching(Aws::S3::S3Client & client, const S3::URI & Pipe StorageS3::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 5a702aa8785..1ecc9409671 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -44,7 +44,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index c4344cf6f1f..5b133ef5e90 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -300,7 +300,7 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Stora Pipe StorageStripeLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index f88120a932e..fb70da24ac8 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -28,7 +28,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index a59480f0a0d..81eec735c8a 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -427,7 +427,7 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const Storage Pipe StorageTinyLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, const size_t max_block_size, diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index dc6ff101503..0c37a16d179 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -27,7 +27,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index a6c8f1b39f8..139d9d08e44 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -180,7 +180,7 @@ std::function IStorageURLBase::getReadPOSTDataCallback( Pipe IStorageURLBase::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 7983ad71520..e4ce87af550 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -22,7 +22,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index 387d2065f92..84845170d36 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -24,7 +24,7 @@ StorageValues::StorageValues( Pipe StorageValues::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, diff --git a/src/Storages/StorageValues.h b/src/Storages/StorageValues.h index 8a1a06eeb54..5729f245149 100644 --- a/src/Storages/StorageValues.h +++ b/src/Storages/StorageValues.h @@ -18,7 +18,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 4b7733c1cd2..f710a1289aa 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -50,7 +50,7 @@ StorageView::StorageView( Pipe StorageView::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, diff --git a/src/Storages/StorageView.h b/src/Storages/StorageView.h index 682c7424b98..56c9b0b4b1f 100644 --- a/src/Storages/StorageView.h +++ b/src/Storages/StorageView.h @@ -24,7 +24,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index 3350a4352db..0b4251bc912 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -87,7 +87,7 @@ std::function StorageXDBC::getReadPOSTDataCallback( Pipe StorageXDBC::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/StorageXDBC.h b/src/Storages/StorageXDBC.h index 508edf22684..d2bd29d7c3d 100644 --- a/src/Storages/StorageXDBC.h +++ b/src/Storages/StorageXDBC.h @@ -18,7 +18,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 81650d669dc..d83a71c2592 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -35,7 +35,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 8a7368d5ace..d55d2250081 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -243,7 +243,7 @@ private: Pipe StorageSystemColumns::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, const size_t max_block_size, diff --git a/src/Storages/System/StorageSystemColumns.h b/src/Storages/System/StorageSystemColumns.h index d90cec763c9..c4f35485612 100644 --- a/src/Storages/System/StorageSystemColumns.h +++ b/src/Storages/System/StorageSystemColumns.h @@ -20,7 +20,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 5a233ed7a33..d926a7c535e 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -33,7 +33,7 @@ StorageSystemDetachedParts::StorageSystemDetachedParts(const StorageID & table_i Pipe StorageSystemDetachedParts::read( const Names & /* column_names */, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, diff --git a/src/Storages/System/StorageSystemDetachedParts.h b/src/Storages/System/StorageSystemDetachedParts.h index c0f1db51642..4c6970dadd6 100644 --- a/src/Storages/System/StorageSystemDetachedParts.h +++ b/src/Storages/System/StorageSystemDetachedParts.h @@ -25,7 +25,7 @@ protected: Pipe read( const Names & /* column_names */, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index ed831927f16..bdd8e2b1861 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -30,7 +30,7 @@ StorageSystemDisks::StorageSystemDisks(const StorageID & table_id_) Pipe StorageSystemDisks::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, diff --git a/src/Storages/System/StorageSystemDisks.h b/src/Storages/System/StorageSystemDisks.h index d2075c3c784..cff05242019 100644 --- a/src/Storages/System/StorageSystemDisks.h +++ b/src/Storages/System/StorageSystemDisks.h @@ -23,7 +23,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/System/StorageSystemNumbers.cpp b/src/Storages/System/StorageSystemNumbers.cpp index af8073e06dc..677e0c02400 100644 --- a/src/Storages/System/StorageSystemNumbers.cpp +++ b/src/Storages/System/StorageSystemNumbers.cpp @@ -125,7 +125,7 @@ StorageSystemNumbers::StorageSystemNumbers(const StorageID & table_id, bool mult Pipe StorageSystemNumbers::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo &, + SelectQueryInfo &, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, diff --git a/src/Storages/System/StorageSystemNumbers.h b/src/Storages/System/StorageSystemNumbers.h index f907f3d5f93..d12c28c1ce2 100644 --- a/src/Storages/System/StorageSystemNumbers.h +++ b/src/Storages/System/StorageSystemNumbers.h @@ -32,7 +32,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/System/StorageSystemOne.cpp b/src/Storages/System/StorageSystemOne.cpp index 57b87e165a9..c456b22e97b 100644 --- a/src/Storages/System/StorageSystemOne.cpp +++ b/src/Storages/System/StorageSystemOne.cpp @@ -23,7 +23,7 @@ StorageSystemOne::StorageSystemOne(const StorageID & table_id_) Pipe StorageSystemOne::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo &, + SelectQueryInfo &, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, diff --git a/src/Storages/System/StorageSystemOne.h b/src/Storages/System/StorageSystemOne.h index 3469d6ccb29..8228ce465e0 100644 --- a/src/Storages/System/StorageSystemOne.h +++ b/src/Storages/System/StorageSystemOne.h @@ -24,7 +24,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index f590244116d..f0fa1251674 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -226,7 +226,7 @@ StoragesInfo StoragesInfoStream::next() Pipe StorageSystemPartsBase::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 2cb19f8f17d..eec6d5ab331 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -58,7 +58,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index f28ecdb7f65..c3ce43f0a79 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -60,7 +60,7 @@ StorageSystemReplicas::StorageSystemReplicas(const StorageID & table_id_) Pipe StorageSystemReplicas::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, diff --git a/src/Storages/System/StorageSystemReplicas.h b/src/Storages/System/StorageSystemReplicas.h index c198cc29ddc..d9e364a28c0 100644 --- a/src/Storages/System/StorageSystemReplicas.h +++ b/src/Storages/System/StorageSystemReplicas.h @@ -21,7 +21,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/System/StorageSystemStoragePolicies.cpp b/src/Storages/System/StorageSystemStoragePolicies.cpp index c8d266644eb..67a876e69bb 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.cpp +++ b/src/Storages/System/StorageSystemStoragePolicies.cpp @@ -38,7 +38,7 @@ StorageSystemStoragePolicies::StorageSystemStoragePolicies(const StorageID & tab Pipe StorageSystemStoragePolicies::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & /*query_info*/, + SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, const size_t /*max_block_size*/, diff --git a/src/Storages/System/StorageSystemStoragePolicies.h b/src/Storages/System/StorageSystemStoragePolicies.h index 15e5e497785..afd5e672d66 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.h +++ b/src/Storages/System/StorageSystemStoragePolicies.h @@ -23,7 +23,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 0ad961ad7d8..cc3c8d4386e 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -488,7 +488,7 @@ private: Pipe StorageSystemTables::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, const size_t max_block_size, diff --git a/src/Storages/System/StorageSystemTables.h b/src/Storages/System/StorageSystemTables.h index 259eb096ea7..2e0b3386f8c 100644 --- a/src/Storages/System/StorageSystemTables.h +++ b/src/Storages/System/StorageSystemTables.h @@ -21,7 +21,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index 270dcc81cc1..ed5ab93369a 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -93,7 +93,7 @@ StorageSystemZeros::StorageSystemZeros(const StorageID & table_id_, bool multith Pipe StorageSystemZeros::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo &, + SelectQueryInfo &, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, diff --git a/src/Storages/System/StorageSystemZeros.h b/src/Storages/System/StorageSystemZeros.h index 41de3ce6246..04733f550c1 100644 --- a/src/Storages/System/StorageSystemZeros.h +++ b/src/Storages/System/StorageSystemZeros.h @@ -23,7 +23,7 @@ public: Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index 1eab072fb4b..d00e1aedb80 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -120,7 +120,7 @@ std::string readData(DB::StoragePtr & table, const DB::Context & context) context, QueryProcessingStage::Complete, query_info); QueryPipeline pipeline; - pipeline.init(table->read(column_names, metadata_snapshot, {}, context, stage, 8192, 1)); + pipeline.init(table->read(column_names, metadata_snapshot, query_info, context, stage, 8192, 1)); BlockInputStreamPtr in = std::make_shared(std::move(pipeline)); Block sample; From cb7db6d232e9ece0c16cf9080c865d9eef7ce1e6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 30 Sep 2020 15:40:46 +0300 Subject: [PATCH 014/314] Split select and process merges (cherry picked from commit 1889eb0ff18b8ba75744dd795ee3339df9ad3e48) --- src/Storages/StorageMergeTree.cpp | 366 ++++++++++---------- src/Storages/StorageMergeTree.h | 18 +- src/Storages/StorageReplicatedMergeTree.cpp | 44 ++- src/Storages/StorageReplicatedMergeTree.h | 5 + 4 files changed, 239 insertions(+), 194 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 0f1afe1bd62..6a2823031f2 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -622,6 +622,100 @@ void StorageMergeTree::loadMutations() increment.value = std::max(Int64(increment.value.load()), current_mutations_by_version.rbegin()->first); } +std::optional StorageMergeTree::selectPartsToMerge(const StorageMetadataPtr &, bool aggressive, const String & partition_id, bool final, String * out_disable_reason) +{ + std::unique_lock lock(currently_processing_in_background_mutex); + auto data_settings = getSettings(); + + FutureMergedMutatedPart future_part; + + /// You must call destructor with unlocked `currently_processing_in_background_mutex`. + CurrentlyMergingPartsTaggerPtr merging_tagger; + MergeList::EntryPtr merge_entry; + + auto can_merge = [this, &lock] (const DataPartPtr & left, const DataPartPtr & right, String *) -> bool + { + /// This predicate is checked for the first part of each partition. + /// (left = nullptr, right = "first part of partition") + if (!left) + return !currently_merging_mutating_parts.count(right); + return !currently_merging_mutating_parts.count(left) && !currently_merging_mutating_parts.count(right) + && getCurrentMutationVersion(left, lock) == getCurrentMutationVersion(right, lock); + }; + + bool selected = false; + + if (partition_id.empty()) + { + UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(); + bool merge_with_ttl_allowed = getTotalMergesWithTTLInMergeList() < data_settings->max_number_of_merges_with_ttl_in_pool; + + /// TTL requirements is much more strict than for regular merge, so + /// if regular not possible, than merge with ttl is not also not + /// possible. + if (max_source_parts_size > 0) + { + selected = merger_mutator.selectPartsToMerge( + future_part, + aggressive, + max_source_parts_size, + can_merge, + merge_with_ttl_allowed, + out_disable_reason); + } + else if (out_disable_reason) + *out_disable_reason = "Current value of max_source_parts_size is zero"; + } + else + { + while (true) + { + UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); + selected = merger_mutator.selectAllPartsToMergeWithinPartition( + future_part, disk_space, can_merge, partition_id, final, out_disable_reason); + + /// If final - we will wait for currently processing merges to finish and continue. + /// TODO Respect query settings for timeout + if (final + && !selected + && !currently_merging_mutating_parts.empty() + && out_disable_reason + && out_disable_reason->empty()) + { + LOG_DEBUG(log, "Waiting for currently running merges ({} parts are merging right now) to perform OPTIMIZE FINAL", + currently_merging_mutating_parts.size()); + + if (std::cv_status::timeout == currently_processing_in_background_condition.wait_for( + lock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC))) + { + *out_disable_reason = "Timeout while waiting for already running merges before running OPTIMIZE with FINAL"; + break; + } + } + else + break; + } + } + + if (!selected) + { + if (out_disable_reason) + { + if (!out_disable_reason->empty()) + { + *out_disable_reason += ". "; + } + *out_disable_reason += "Cannot select parts for optimization"; + } + + return {}; + } + + merging_tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, false); + auto table_id = getStorageID(); + merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); + return MergeMutateSelectedEntry{future_part, std::move(merging_tagger), std::move(merge_entry), {}}; +} bool StorageMergeTree::merge( bool aggressive, @@ -632,99 +726,12 @@ bool StorageMergeTree::merge( { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto metadata_snapshot = getInMemoryMetadataPtr(); - auto data_settings = getSettings(); - FutureMergedMutatedPart future_part; - - /// You must call destructor with unlocked `currently_processing_in_background_mutex`. - std::optional merging_tagger; - MergeList::EntryPtr merge_entry; - - { - std::unique_lock lock(currently_processing_in_background_mutex); - - auto can_merge = [this, &lock] (const DataPartPtr & left, const DataPartPtr & right, String *) -> bool - { - /// This predicate is checked for the first part of each partition. - /// (left = nullptr, right = "first part of partition") - if (!left) - return !currently_merging_mutating_parts.count(right); - return !currently_merging_mutating_parts.count(left) && !currently_merging_mutating_parts.count(right) - && getCurrentMutationVersion(left, lock) == getCurrentMutationVersion(right, lock); - }; - - bool selected = false; - - if (partition_id.empty()) - { - UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(); - bool merge_with_ttl_allowed = getTotalMergesWithTTLInMergeList() < data_settings->max_number_of_merges_with_ttl_in_pool; - - /// TTL requirements is much more strict than for regular merge, so - /// if regular not possible, than merge with ttl is not also not - /// possible. - if (max_source_parts_size > 0) - { - selected = merger_mutator.selectPartsToMerge( - future_part, - aggressive, - max_source_parts_size, - can_merge, - merge_with_ttl_allowed, - out_disable_reason); - } - else if (out_disable_reason) - *out_disable_reason = "Current value of max_source_parts_size is zero"; - } - else - { - while (true) - { - UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); - selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_part, disk_space, can_merge, partition_id, final, out_disable_reason); - - /// If final - we will wait for currently processing merges to finish and continue. - /// TODO Respect query settings for timeout - if (final - && !selected - && !currently_merging_mutating_parts.empty() - && out_disable_reason - && out_disable_reason->empty()) - { - LOG_DEBUG(log, "Waiting for currently running merges ({} parts are merging right now) to perform OPTIMIZE FINAL", - currently_merging_mutating_parts.size()); - - if (std::cv_status::timeout == currently_processing_in_background_condition.wait_for( - lock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC))) - { - *out_disable_reason = "Timeout while waiting for already running merges before running OPTIMIZE with FINAL"; - break; - } - } - else - break; - } - } - - if (!selected) - { - if (out_disable_reason) - { - if (!out_disable_reason->empty()) - { - *out_disable_reason += ". "; - } - *out_disable_reason += "Cannot select parts for optimization"; - } - return false; - } - - merging_tagger.emplace(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, metadata_snapshot, false); - auto table_id = getStorageID(); - merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); - } + auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason); + if (!merge_mutate_entry) + return false; + auto & future_part = merge_mutate_entry->future_part; /// Logging Stopwatch stopwatch; MutableDataPartPtr new_part; @@ -738,14 +745,14 @@ bool StorageMergeTree::merge( future_part.name, new_part, future_part.parts, - merge_entry.get()); + merge_mutate_entry->merge_entry.get()); }; try { new_part = merger_mutator.mergePartsToTemporaryPart( - future_part, metadata_snapshot, *merge_entry, table_lock_holder, time(nullptr), global_context, - merging_tagger->reserved_space, deduplicate); + future_part, metadata_snapshot, *(merge_mutate_entry->merge_entry), table_lock_holder, time(nullptr), + merge_mutate_entry->tagger->reserved_space, deduplicate); merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr); write_part_log({}); @@ -783,95 +790,98 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::movePartsTask() } } +std::optional StorageMergeTree::selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String */* disable_reason */) +{ + std::lock_guard lock(currently_processing_in_background_mutex); + size_t max_ast_elements = global_context.getSettingsRef().max_expanded_ast_elements; + + FutureMergedMutatedPart future_part; + MutationCommands commands; + + CurrentlyMergingPartsTaggerPtr tagger; + + if (current_mutations_by_version.empty()) + return {}; + + auto mutations_end_it = current_mutations_by_version.end(); + for (const auto & part : getDataPartsVector()) + { + if (currently_merging_mutating_parts.count(part)) + continue; + + auto mutations_begin_it = current_mutations_by_version.upper_bound(part->info.getDataVersion()); + if (mutations_begin_it == mutations_end_it) + continue; + + size_t max_source_part_size = merger_mutator.getMaxSourcePartSizeForMutation(); + if (max_source_part_size < part->getBytesOnDisk()) + { + LOG_DEBUG(log, "Current max source part size for mutation is {} but part size {}. Will not mutate part {}. " + "Max size depends not only on available space, but also on settings " + "'number_of_free_entries_in_pool_to_execute_mutation' and 'background_pool_size'", + max_source_part_size, part->getBytesOnDisk(), part->name); + continue; + } + + size_t current_ast_elements = 0; + for (auto it = mutations_begin_it; it != mutations_end_it; ++it) + { + size_t commands_size = 0; + MutationCommands commands_for_size_validation; + for (const auto & command : it->second.commands) + { + if (command.type != MutationCommand::Type::DROP_COLUMN + && command.type != MutationCommand::Type::DROP_INDEX + && command.type != MutationCommand::Type::RENAME_COLUMN) + { + commands_for_size_validation.push_back(command); + } + else + { + commands_size += command.ast->size(); + } + } + + if (!commands_for_size_validation.empty()) + { + MutationsInterpreter interpreter( + shared_from_this(), metadata_snapshot, commands_for_size_validation, global_context, false); + commands_size += interpreter.evaluateCommandsSize(); + } + + if (current_ast_elements + commands_size >= max_ast_elements) + break; + + current_ast_elements += commands_size; + commands.insert(commands.end(), it->second.commands.begin(), it->second.commands.end()); + } + + auto new_part_info = part->info; + new_part_info.mutation = current_mutations_by_version.rbegin()->first; + + future_part.parts.push_back(part); + future_part.part_info = new_part_info; + future_part.name = part->getNewName(new_part_info); + future_part.type = part->getType(); + + tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, true); + auto table_id = getStorageID(); + MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); + return MergeMutateSelectedEntry{future_part, std::move(tagger), std::move(merge_entry), commands}; + } + return {}; +} bool StorageMergeTree::tryMutatePart() { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); StorageMetadataPtr metadata_snapshot = getInMemoryMetadataPtr(); - size_t max_ast_elements = global_context.getSettingsRef().max_expanded_ast_elements; - FutureMergedMutatedPart future_part; - MutationCommands commands; - /// You must call destructor with unlocked `currently_processing_in_background_mutex`. - std::optional tagger; - { - std::lock_guard lock(currently_processing_in_background_mutex); - - if (current_mutations_by_version.empty()) - return false; - - auto mutations_end_it = current_mutations_by_version.end(); - for (const auto & part : getDataPartsVector()) - { - if (currently_merging_mutating_parts.count(part)) - continue; - - auto mutations_begin_it = current_mutations_by_version.upper_bound(part->info.getDataVersion()); - if (mutations_begin_it == mutations_end_it) - continue; - - size_t max_source_part_size = merger_mutator.getMaxSourcePartSizeForMutation(); - if (max_source_part_size < part->getBytesOnDisk()) - { - LOG_DEBUG(log, "Current max source part size for mutation is {} but part size {}. Will not mutate part {}. " - "Max size depends not only on available space, but also on settings " - "'number_of_free_entries_in_pool_to_execute_mutation' and 'background_pool_size'", - max_source_part_size, part->getBytesOnDisk(), part->name); - continue; - } - - size_t current_ast_elements = 0; - for (auto it = mutations_begin_it; it != mutations_end_it; ++it) - { - size_t commands_size = 0; - MutationCommands commands_for_size_validation; - for (const auto & command : it->second.commands) - { - if (command.type != MutationCommand::Type::DROP_COLUMN - && command.type != MutationCommand::Type::DROP_INDEX - && command.type != MutationCommand::Type::RENAME_COLUMN) - { - commands_for_size_validation.push_back(command); - } - else - { - commands_size += command.ast->size(); - } - } - - if (!commands_for_size_validation.empty()) - { - MutationsInterpreter interpreter( - shared_from_this(), metadata_snapshot, commands_for_size_validation, global_context, false); - commands_size += interpreter.evaluateCommandsSize(); - } - - if (current_ast_elements + commands_size >= max_ast_elements) - break; - - current_ast_elements += commands_size; - commands.insert(commands.end(), it->second.commands.begin(), it->second.commands.end()); - } - - auto new_part_info = part->info; - new_part_info.mutation = current_mutations_by_version.rbegin()->first; - - future_part.parts.push_back(part); - future_part.part_info = new_part_info; - future_part.name = part->getNewName(new_part_info); - future_part.type = part->getType(); - - tagger.emplace(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); - break; - } - } - - if (!tagger) + auto merge_mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr); + if (!merge_mutate_entry) return false; - auto table_id = getStorageID(); - MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); - + auto & future_part = merge_mutate_entry->future_part; Stopwatch stopwatch; MutableDataPartPtr new_part; @@ -884,14 +894,14 @@ bool StorageMergeTree::tryMutatePart() future_part.name, new_part, future_part.parts, - merge_entry.get()); + merge_mutate_entry->merge_entry.get()); }; try { new_part = merger_mutator.mutatePartToTemporaryPart( - future_part, metadata_snapshot, commands, *merge_entry, - time(nullptr), global_context, tagger->reserved_space, table_lock_holder); + future_part, metadata_snapshot, merge_mutate_entry->commands, *(merge_mutate_entry->merge_entry), + time(nullptr), global_context, merge_mutate_entry->tagger->reserved_space, table_lock_holder); renameTempPartAndReplace(new_part); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 5662f9e0088..9b740f575c2 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -21,6 +21,8 @@ namespace DB { +struct CurrentlyMergingPartsTagger; + /** See the description of the data structure in MergeTreeData. */ class StorageMergeTree final : public ext::shared_ptr_helper, public MergeTreeData @@ -140,6 +142,20 @@ private: /// Try and find a single part to mutate and mutate it. If some part was successfully mutated, return true. bool tryMutatePart(); + friend struct CurrentlyMergingPartsTagger; + + using CurrentlyMergingPartsTaggerPtr = std::unique_ptr; + + struct MergeMutateSelectedEntry + { + FutureMergedMutatedPart future_part; + CurrentlyMergingPartsTaggerPtr tagger; + MergeList::EntryPtr merge_entry; + MutationCommands commands; + }; + + std::optional selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * disable_reason); + std::optional selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason); BackgroundProcessingPoolTaskResult mergeMutateTask(); @@ -173,7 +189,7 @@ private: friend class MergeTreeBlockOutputStream; friend class MergeTreeData; - friend struct CurrentlyMergingPartsTagger; + protected: diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2ac8ddb7846..ae30977a9f2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2537,16 +2537,8 @@ void StorageReplicatedMergeTree::mutationsUpdatingTask() } } - -BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask() +ReplicatedMergeTreeQueue::SelectedEntry StorageReplicatedMergeTree::selectQueueEntry() { - /// If replication queue is stopped exit immediately as we successfully executed the task - if (queue.actions_blocker.isCancelled()) - { - std::this_thread::sleep_for(std::chrono::milliseconds(5)); - return BackgroundProcessingPoolTaskResult::SUCCESS; - } - /// This object will mark the element of the queue as running. ReplicatedMergeTreeQueue::SelectedEntry selected; @@ -2559,14 +2551,14 @@ BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask() tryLogCurrentException(log, __PRETTY_FUNCTION__); } - LogEntryPtr & entry = selected.first; + return selected; +} - if (!entry) - return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO; +bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntry & selected_entry) +{ - time_t prev_attempt_time = entry->last_attempt_time; - - bool res = queue.processEntry([this]{ return getZooKeeper(); }, entry, [&](LogEntryPtr & entry_to_process) + LogEntryPtr & entry = selected_entry.first; + return queue.processEntry([this]{ return getZooKeeper(); }, entry, [&](LogEntryPtr & entry_to_process) { try { @@ -2605,6 +2597,28 @@ BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask() throw; } }); +} + +BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask() +{ + /// If replication queue is stopped exit immediately as we successfully executed the task + if (queue.actions_blocker.isCancelled()) + { + std::this_thread::sleep_for(std::chrono::milliseconds(5)); + return BackgroundProcessingPoolTaskResult::SUCCESS; + } + + /// This object will mark the element of the queue as running. + ReplicatedMergeTreeQueue::SelectedEntry selected_entry = selectQueueEntry(); + + LogEntryPtr & entry = selected_entry.first; + + if (!entry) + return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO; + + time_t prev_attempt_time = entry->last_attempt_time; + + bool res = processQueueEntry(selected_entry); /// We will go to sleep if the processing fails and if we have already processed this record recently. bool need_sleep = !res && (entry->last_attempt_time - prev_attempt_time < 10); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 0356cccd302..bb13cd7f230 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -416,6 +416,11 @@ private: /// Clone replica if it is lost. void cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zookeeper); + + ReplicatedMergeTreeQueue::SelectedEntry selectQueueEntry(); + + bool processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntry & entry); + /** Performs actions from the queue. */ BackgroundProcessingPoolTaskResult queueTask(); From 48aaa0ce160b73b3ba83bdff7bd9cfaf55939191 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 30 Sep 2020 16:49:22 +0300 Subject: [PATCH 015/314] Split merge to separate method (cherry picked from commit 14a7a25c4b5d52134349300091df034ab6e652a4) --- src/Storages/StorageMergeTree.cpp | 34 +++++++++++++++++++------------ src/Storages/StorageMergeTree.h | 3 +++ 2 files changed, 24 insertions(+), 13 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 6a2823031f2..cf800dc0d50 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -624,6 +624,7 @@ void StorageMergeTree::loadMutations() std::optional StorageMergeTree::selectPartsToMerge(const StorageMetadataPtr &, bool aggressive, const String & partition_id, bool final, String * out_disable_reason) { + auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); std::unique_lock lock(currently_processing_in_background_mutex); auto data_settings = getSettings(); @@ -724,15 +725,18 @@ bool StorageMergeTree::merge( bool deduplicate, String * out_disable_reason) { - auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto metadata_snapshot = getInMemoryMetadataPtr(); auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason); if (!merge_mutate_entry) return false; - auto & future_part = merge_mutate_entry->future_part; - /// Logging + return mergeSelectedParts(metadata_snapshot, deduplicate, *merge_mutate_entry); +} + +bool StorageMergeTree::mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & merge_mutate_entry) { + auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + auto & future_part = merge_mutate_entry.future_part; Stopwatch stopwatch; MutableDataPartPtr new_part; @@ -745,14 +749,14 @@ bool StorageMergeTree::merge( future_part.name, new_part, future_part.parts, - merge_mutate_entry->merge_entry.get()); + merge_mutate_entry.merge_entry.get()); }; try { new_part = merger_mutator.mergePartsToTemporaryPart( - future_part, metadata_snapshot, *(merge_mutate_entry->merge_entry), table_lock_holder, time(nullptr), - merge_mutate_entry->tagger->reserved_space, deduplicate); + future_part, metadata_snapshot, *(merge_mutate_entry.merge_entry), table_lock_holder, time(nullptr), + merge_mutate_entry.tagger->reserved_space, deduplicate); merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr); write_part_log({}); @@ -766,7 +770,6 @@ bool StorageMergeTree::merge( return true; } - bool StorageMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & part) const { std::lock_guard background_processing_lock(currently_processing_in_background_mutex); @@ -792,6 +795,7 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::movePartsTask() std::optional StorageMergeTree::selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String */* disable_reason */) { + auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); std::lock_guard lock(currently_processing_in_background_mutex); size_t max_ast_elements = global_context.getSettingsRef().max_expanded_ast_elements; @@ -874,14 +878,19 @@ std::optional StorageMergeTree::sele bool StorageMergeTree::tryMutatePart() { - auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); StorageMetadataPtr metadata_snapshot = getInMemoryMetadataPtr(); auto merge_mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr); if (!merge_mutate_entry) return false; - auto & future_part = merge_mutate_entry->future_part; + return mutateSelectedPart(metadata_snapshot, *merge_mutate_entry); +} + +bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & merge_mutate_entry) +{ + auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + auto & future_part = merge_mutate_entry.future_part; Stopwatch stopwatch; MutableDataPartPtr new_part; @@ -894,14 +903,14 @@ bool StorageMergeTree::tryMutatePart() future_part.name, new_part, future_part.parts, - merge_mutate_entry->merge_entry.get()); + merge_mutate_entry.merge_entry.get()); }; try { new_part = merger_mutator.mutatePartToTemporaryPart( - future_part, metadata_snapshot, merge_mutate_entry->commands, *(merge_mutate_entry->merge_entry), - time(nullptr), global_context, merge_mutate_entry->tagger->reserved_space, table_lock_holder); + future_part, metadata_snapshot, merge_mutate_entry.commands, *(merge_mutate_entry.merge_entry), + time(nullptr), global_context, merge_mutate_entry.tagger->reserved_space, table_lock_holder); renameTempPartAndReplace(new_part); @@ -918,7 +927,6 @@ bool StorageMergeTree::tryMutatePart() return true; } - BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask() { if (shutdown_called) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 9b740f575c2..ac6939bd900 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -155,7 +155,10 @@ private: }; std::optional selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * disable_reason); + bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry); + std::optional selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason); + bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry); BackgroundProcessingPoolTaskResult mergeMutateTask(); From ef355e71d6026f43df5879fd22438384a25db112 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 13 Oct 2020 15:10:46 +0300 Subject: [PATCH 016/314] Buildable code --- src/Storages/StorageMergeTree.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index cf800dc0d50..48563fc0c40 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -622,7 +622,7 @@ void StorageMergeTree::loadMutations() increment.value = std::max(Int64(increment.value.load()), current_mutations_by_version.rbegin()->first); } -std::optional StorageMergeTree::selectPartsToMerge(const StorageMetadataPtr &, bool aggressive, const String & partition_id, bool final, String * out_disable_reason) +std::optional StorageMergeTree::selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason) { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); std::unique_lock lock(currently_processing_in_background_mutex); @@ -712,7 +712,7 @@ std::optional StorageMergeTree::sele return {}; } - merging_tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, false); + merging_tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, metadata_snapshot, false); auto table_id = getStorageID(); merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); return MergeMutateSelectedEntry{future_part, std::move(merging_tagger), std::move(merge_entry), {}}; @@ -756,7 +756,7 @@ bool StorageMergeTree::mergeSelectedParts(const StorageMetadataPtr & metadata_sn { new_part = merger_mutator.mergePartsToTemporaryPart( future_part, metadata_snapshot, *(merge_mutate_entry.merge_entry), table_lock_holder, time(nullptr), - merge_mutate_entry.tagger->reserved_space, deduplicate); + global_context, merge_mutate_entry.tagger->reserved_space, deduplicate); merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr); write_part_log({}); @@ -868,7 +868,7 @@ std::optional StorageMergeTree::sele future_part.name = part->getNewName(new_part_info); future_part.type = part->getType(); - tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, true); + tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); auto table_id = getStorageID(); MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); return MergeMutateSelectedEntry{future_part, std::move(tagger), std::move(merge_entry), commands}; From dcf489ac8f8887588cd973a955abe916dabfaca8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 30 Sep 2020 16:56:44 +0300 Subject: [PATCH 017/314] Remove unused methods (cherry picked from commit 77f5841a46016a05d757c2415d3b881390c8ba7a) --- src/Storages/StorageMergeTree.cpp | 18 +++++------------- src/Storages/StorageMergeTree.h | 2 -- 2 files changed, 5 insertions(+), 15 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 48563fc0c40..b68ce9be37b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -876,17 +876,6 @@ std::optional StorageMergeTree::sele return {}; } -bool StorageMergeTree::tryMutatePart() -{ - StorageMetadataPtr metadata_snapshot = getInMemoryMetadataPtr(); - - auto merge_mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr); - if (!merge_mutate_entry) - return false; - - return mutateSelectedPart(metadata_snapshot, *merge_mutate_entry); -} - bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & merge_mutate_entry) { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); @@ -951,11 +940,14 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask() clearOldMutations(); } + auto metadata_snapshot = getInMemoryMetadataPtr(); + auto merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr); ///TODO: read deduplicate option from table config - if (merge(false /*aggressive*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/)) + if (merge_entry && mergeSelectedParts(metadata_snapshot, false, *merge_entry)) return BackgroundProcessingPoolTaskResult::SUCCESS; - if (tryMutatePart()) + auto mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr); + if (mutate_entry && mutateSelectedPart(metadata_snapshot, *mutate_entry)) return BackgroundProcessingPoolTaskResult::SUCCESS; return BackgroundProcessingPoolTaskResult::ERROR; diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index ac6939bd900..519352a0a8a 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -140,8 +140,6 @@ private: /// Wait until mutation with version will finish mutation for all parts void waitForMutation(Int64 version, const String & file_name); - /// Try and find a single part to mutate and mutate it. If some part was successfully mutated, return true. - bool tryMutatePart(); friend struct CurrentlyMergingPartsTagger; using CurrentlyMergingPartsTaggerPtr = std::unique_ptr; From 163d33fd214013e47a17f68db69a9f86d82c2777 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 9 Oct 2020 16:34:44 +0300 Subject: [PATCH 018/314] Add perftest for merges (cherry picked from commit e655f009d2f171b30feb34da2bc25706a0e3f91e) --- tests/performance/insert_sequential.xml | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 tests/performance/insert_sequential.xml diff --git a/tests/performance/insert_sequential.xml b/tests/performance/insert_sequential.xml new file mode 100644 index 00000000000..0c98c04ec8b --- /dev/null +++ b/tests/performance/insert_sequential.xml @@ -0,0 +1,11 @@ + + + + 1 + 1 + 1 + + CREATE TABLE t (x UInt64) ENGINE = MergeTree() ORDER BY tuple() + INSERT INTO t SELECT * FROM numbers(20000) + DROP TABLE IF EXISTS t + From fd35368c5911a84c40ac2b12e55d86c84c1d3517 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 13 Oct 2020 17:25:42 +0300 Subject: [PATCH 019/314] First simple implementation --- .../MergeTree/BackgroundJobsExecutor.cpp | 49 ++++++++++ .../MergeTree/BackgroundJobsExecutor.h | 34 +++++++ .../MergeTree/MergeTreeBackgroundJob.h | 46 ++++++++++ .../MergeTree/MergeTreeBlockOutputStream.cpp | 7 +- src/Storages/MergeTree/MergeTreeData.h | 3 + .../MergeTree/MergeTreeDataMergerMutator.cpp | 3 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.h | 2 +- src/Storages/StorageMergeTree.cpp | 92 ++++++++++++++----- src/Storages/StorageMergeTree.h | 7 +- src/Storages/StorageReplicatedMergeTree.cpp | 25 ++++- src/Storages/StorageReplicatedMergeTree.h | 2 + 12 files changed, 238 insertions(+), 34 deletions(-) create mode 100644 src/Storages/MergeTree/BackgroundJobsExecutor.cpp create mode 100644 src/Storages/MergeTree/BackgroundJobsExecutor.h create mode 100644 src/Storages/MergeTree/MergeTreeBackgroundJob.h diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp new file mode 100644 index 00000000000..662fa71d318 --- /dev/null +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -0,0 +1,49 @@ +#include + +namespace DB +{ + +BackgroundJobsExecutor::BackgroundJobsExecutor( + MergeTreeData & data_, + Context & global_context) + : data(data_) + , data_processing_pool(global_context.getSettingsRef().background_pool_size, 0, 10000, false) + , move_pool(global_context.getSettingsRef().background_move_pool_size, 0, 10000, false) +{ + data_processing_task = global_context.getSchedulePool().createTask( + data.getStorageID().getFullTableName() + " (dataProcessingTask)", [this]{ dataProcessingTask(); }); +} + +void BackgroundJobsExecutor::dataProcessingTask() +try +{ + auto job = data.getDataProcessingJob(); + if (job) + data_processing_pool.scheduleOrThrowOnError(*job); + + data_processing_task->schedule(); +} +catch (...) +{ + tryLogCurrentException(__PRETTY_FUNCTION__); +} + +void BackgroundJobsExecutor::start() +{ + if (data_processing_task) + data_processing_task->activateAndSchedule(); +} + +void BackgroundJobsExecutor::triggerDataProcessing() +{ + if (data_processing_task) + data_processing_task->schedule(); +} + +void BackgroundJobsExecutor::finish() +{ + data_processing_task->deactivate(); + data_processing_pool.wait(); +} + +} diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h new file mode 100644 index 00000000000..aa166eb4d73 --- /dev/null +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -0,0 +1,34 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class BackgroundJobsExecutor +{ +private: + MergeTreeData & data; + ThreadPool data_processing_pool; + ThreadPool move_pool; + + BackgroundSchedulePool::TaskHolder data_processing_task; + BackgroundSchedulePool::TaskHolder move_processing_task; + + void dataProcessingTask(); + +public: + BackgroundJobsExecutor( + MergeTreeData & data_, + Context & global_context_); + + void triggerDataProcessing(); + void triggerMovesProcessing(); + void start(); + void finish(); +}; + +} diff --git a/src/Storages/MergeTree/MergeTreeBackgroundJob.h b/src/Storages/MergeTree/MergeTreeBackgroundJob.h new file mode 100644 index 00000000000..2f432114db1 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeBackgroundJob.h @@ -0,0 +1,46 @@ +# pragma once +#include +#include +#include +#include + +namespace DB +{ + +enum PoolType +{ + MERGE_MUTATE, + MOVE, + FETCH, +}; + +struct MergeTreeBackgroundJob +{ + ThreadPool::Job job; + CurrentMetrics::Metric metric; + PoolType execute_in_pool; + + MergeTreeBackgroundJob(ThreadPool::Job && job_, CurrentMetrics::Metric metric_, PoolType execute_in_pool_) + : job(std::move(job_)), metric(metric_), execute_in_pool(execute_in_pool_) + {} + + void operator()() + try + { + if (metric != 0) + { + CurrentMetrics::Increment metric_increment{metric}; + job(); + } + else + { + job(); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +}; + +} diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index 4269abe2655..5ad6a7eebc1 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -27,11 +27,8 @@ void MergeTreeBlockOutputStream::write(const Block & block) PartLog::addNewPart(storage.global_context, part, watch.elapsed()); - if (storage.merging_mutating_task_handle) - { - /// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'. - storage.merging_mutating_task_handle->signalReadyToRun(); - } + /// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'. + storage.background_executor.triggerDataProcessing(); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 5c18661dad1..8c5333315fc 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -25,6 +25,7 @@ #include #include #include +#include #include #include @@ -710,6 +711,8 @@ public: /// Mutex for currently_moving_parts mutable std::mutex moving_parts_mutex; + virtual std::optional getDataProcessingJob() { return {}; } + protected: friend class IMergeTreeDataPart; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index fb0a488700c..942bac0d294 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -154,7 +154,6 @@ MergeTreeDataMergerMutator::MergeTreeDataMergerMutator(MergeTreeData & data_, si { } - UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge() const { size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); @@ -166,7 +165,9 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge() const UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used) const { if (pool_used > pool_size) + { throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR); + } size_t free_entries = pool_size - pool_used; const auto data_settings = data.getSettings(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 45e16e81208..48caf59e7ba 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1287,7 +1287,7 @@ ReplicatedMergeTreeQueue::SelectedEntry ReplicatedMergeTreeQueue::selectEntryToP } if (entry) - return { entry, std::unique_ptr{ new CurrentlyExecuting(entry, *this) } }; + return { entry, std::shared_ptr{ new CurrentlyExecuting(entry, *this) } }; else return {}; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 88a61f50225..c72569a5071 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -319,7 +319,7 @@ public: /** Select the next action to process. * merger_mutator is used only to check if the merges are not suspended. */ - using SelectedEntry = std::pair>; + using SelectedEntry = std::pair>; SelectedEntry selectEntryToProcess(MergeTreeDataMergerMutator & merger_mutator, MergeTreeData & data); /** Execute `func` function to handle the action. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b68ce9be37b..00ddb7a57ce 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -27,6 +27,10 @@ #include #include +namespace CurrentMetrics +{ + extern const Metric BackgroundPoolTask; +} namespace DB { @@ -73,7 +77,8 @@ StorageMergeTree::StorageMergeTree( attach) , reader(*this) , writer(*this) - , merger_mutator(*this, global_context.getBackgroundPool().getNumberOfThreads()) + , merger_mutator(*this, global_context.getSettingsRef().background_pool_size) + , background_executor(*this, global_context) { loadDataParts(has_force_restore_data_flag); @@ -100,11 +105,7 @@ void StorageMergeTree::startup() try { - auto & merge_pool = global_context.getBackgroundPool(); - merging_mutating_task_handle = merge_pool.createTask([this] { return mergeMutateTask(); }); - /// Ensure that thread started only after assignment to 'merging_mutating_task_handle' is done. - merge_pool.startTask(merging_mutating_task_handle); - + background_executor.start(); startBackgroundMovesIfNeeded(); } catch (...) @@ -142,8 +143,7 @@ void StorageMergeTree::shutdown() merger_mutator.merges_blocker.cancelForever(); parts_mover.moves_blocker.cancelForever(); - if (merging_mutating_task_handle) - global_context.getBackgroundPool().removeTask(merging_mutating_task_handle); + background_executor.finish(); if (moving_task_handle) global_context.getBackgroundMovePool().removeTask(moving_task_handle); @@ -361,7 +361,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, String current_mutations_by_version.emplace(version, insertion.first->second); LOG_INFO(log, "Added mutation: {}", mutation_file_name); - merging_mutating_task_handle->signalReadyToRun(); + background_executor.triggerDataProcessing(); return version; } @@ -591,7 +591,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) } /// Maybe there is another mutation that was blocked by the killed one. Try to execute it immediately. - merging_mutating_task_handle->signalReadyToRun(); + background_executor.triggerDataProcessing(); return CancellationCode::CancelSent; } @@ -712,10 +712,8 @@ std::optional StorageMergeTree::sele return {}; } - merging_tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, metadata_snapshot, false); - auto table_id = getStorageID(); - merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); - return MergeMutateSelectedEntry{future_part, std::move(merging_tagger), std::move(merge_entry), {}}; + merging_tagger = std::make_shared(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, metadata_snapshot, false); + return MergeMutateSelectedEntry{future_part, std::move(merging_tagger), {}}; } bool StorageMergeTree::merge( @@ -739,6 +737,9 @@ bool StorageMergeTree::mergeSelectedParts(const StorageMetadataPtr & metadata_sn auto & future_part = merge_mutate_entry.future_part; Stopwatch stopwatch; MutableDataPartPtr new_part; + auto table_id = getStorageID(); + + auto merge_list_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); auto write_part_log = [&] (const ExecutionStatus & execution_status) { @@ -749,13 +750,13 @@ bool StorageMergeTree::mergeSelectedParts(const StorageMetadataPtr & metadata_sn future_part.name, new_part, future_part.parts, - merge_mutate_entry.merge_entry.get()); + merge_list_entry.get()); }; try { new_part = merger_mutator.mergePartsToTemporaryPart( - future_part, metadata_snapshot, *(merge_mutate_entry.merge_entry), table_lock_holder, time(nullptr), + future_part, metadata_snapshot, *(merge_list_entry), table_lock_holder, time(nullptr), global_context, merge_mutate_entry.tagger->reserved_space, deduplicate); merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr); @@ -868,10 +869,8 @@ std::optional StorageMergeTree::sele future_part.name = part->getNewName(new_part_info); future_part.type = part->getType(); - tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); - auto table_id = getStorageID(); - MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); - return MergeMutateSelectedEntry{future_part, std::move(tagger), std::move(merge_entry), commands}; + tagger = std::make_shared(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); + return MergeMutateSelectedEntry{future_part, std::move(tagger), commands}; } return {}; } @@ -880,6 +879,9 @@ bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_sn { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto & future_part = merge_mutate_entry.future_part; + auto table_id = getStorageID(); + + auto merge_list_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); Stopwatch stopwatch; MutableDataPartPtr new_part; @@ -892,13 +894,13 @@ bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_sn future_part.name, new_part, future_part.parts, - merge_mutate_entry.merge_entry.get()); + merge_list_entry.get()); }; try { new_part = merger_mutator.mutatePartToTemporaryPart( - future_part, metadata_snapshot, merge_mutate_entry.commands, *(merge_mutate_entry.merge_entry), + future_part, metadata_snapshot, merge_mutate_entry.commands, *(merge_list_entry), time(nullptr), global_context, merge_mutate_entry.tagger->reserved_space, table_lock_holder); renameTempPartAndReplace(new_part); @@ -916,6 +918,52 @@ bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_sn return true; } +std::optional StorageMergeTree::getDataProcessingJob() +{ + if (shutdown_called) + return {}; + + if (merger_mutator.merges_blocker.isCancelled()) + return {}; + + auto metadata_snapshot = getInMemoryMetadataPtr(); + std::optional merge_entry, mutate_entry; + + merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr); + if (!merge_entry) + mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr); + + if (merge_entry || mutate_entry) + { + auto job = [this, metadata_snapshot, merge_entry{std::move(merge_entry)}, mutate_entry{std::move(mutate_entry)}] () mutable + { + if (merge_entry) + mergeSelectedParts(metadata_snapshot, false, *merge_entry); + else if (mutate_entry) + mutateSelectedPart(metadata_snapshot, *mutate_entry); + }; + return std::make_optional(std::move(job), CurrentMetrics::BackgroundPoolTask, PoolType::MERGE_MUTATE); + } + else if (auto lock = time_after_previous_cleanup.compareAndRestartDeferred(1)) + { + auto job = [this] () + { + { + auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + /// All use relative_data_path which changes during rename + /// so execute under share lock. + clearOldPartsFromFilesystem(); + clearOldTemporaryDirectories(); + clearOldWriteAheadLogs(); + } + clearOldMutations(); + }; + + return std::make_optional(std::move(job), 0, PoolType::MERGE_MUTATE); + } + return {}; +} + BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask() { if (shutdown_called) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 519352a0a8a..c028e15416f 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -87,6 +88,7 @@ public: CheckResults checkData(const ASTPtr & query, const Context & context) override; + std::optional getDataProcessingJob() override; private: /// Mutex and condvar for synchronous mutations wait @@ -119,7 +121,7 @@ private: std::atomic shutdown_called {false}; /// Task handler for merges, mutations and moves. - BackgroundProcessingPool::TaskHandle merging_mutating_task_handle; + BackgroundJobsExecutor background_executor; BackgroundProcessingPool::TaskHandle moving_task_handle; void loadMutations(); @@ -142,13 +144,12 @@ private: friend struct CurrentlyMergingPartsTagger; - using CurrentlyMergingPartsTaggerPtr = std::unique_ptr; + using CurrentlyMergingPartsTaggerPtr = std::shared_ptr; struct MergeMutateSelectedEntry { FutureMergedMutatedPart future_part; CurrentlyMergingPartsTaggerPtr tagger; - MergeList::EntryPtr merge_entry; MutationCommands commands; }; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ae30977a9f2..ae7ad4a3518 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -194,7 +194,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , replica_path(zookeeper_path + "/replicas/" + replica_name) , reader(*this) , writer(*this) - , merger_mutator(*this, global_context.getBackgroundPool().getNumberOfThreads()) + , merger_mutator(*this, global_context.getSettingsRef().background_pool_size) , queue(*this) , fetcher(*this) , cleanup_thread(*this) @@ -2599,6 +2599,29 @@ bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::Sel }); } + +std::optional StorageReplicatedMergeTree::getDataProcessingJob() +{ + /// If replication queue is stopped exit immediately as we successfully executed the task + if (queue.actions_blocker.isCancelled()) + return {}; + + /// This object will mark the element of the queue as running. + ReplicatedMergeTreeQueue::SelectedEntry selected_entry = selectQueueEntry(); + + LogEntryPtr & entry = selected_entry.first; + + if (!entry) + return {}; + + auto job = [this, selected_entry{std::move(selected_entry)}] () mutable + { + processQueueEntry(selected_entry); + }; + + return std::make_optional(std::move(job), CurrentMetrics::BackgroundPoolTask, PoolType::MERGE_MUTATE); +} + BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask() { /// If replication queue is stopped exit immediately as we successfully executed the task diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index bb13cd7f230..1b65ffdbc25 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -195,6 +195,8 @@ public: */ static void dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica, Poco::Logger * logger); + std::optional getDataProcessingJob() override; + private: /// Get a sequential consistent view of current parts. From efd3126b5d7979da7ff79f380fa2ee46c2d54c36 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 10:22:48 +0300 Subject: [PATCH 020/314] Moving pool --- .../MergeTree/BackgroundJobsExecutor.cpp | 34 +++++++++++++++- .../MergeTree/BackgroundJobsExecutor.h | 5 ++- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 40 ++++++++++++++----- src/Storages/MergeTree/MergeTreeData.h | 14 ++++--- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 + src/Storages/MergeTree/MergeTreePartsMover.h | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 11 +++-- src/Storages/StorageMergeTree.cpp | 21 +++------- src/Storages/StorageMergeTree.h | 5 +-- src/Storages/StorageReplicatedMergeTree.cpp | 24 +++-------- src/Storages/StorageReplicatedMergeTree.h | 7 ++-- 13 files changed, 99 insertions(+), 70 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 662fa71d318..081ef818dcf 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -1,12 +1,14 @@ #include +#include namespace DB { BackgroundJobsExecutor::BackgroundJobsExecutor( MergeTreeData & data_, - Context & global_context) + Context & global_context_) : data(data_) + , global_context(global_context_) , data_processing_pool(global_context.getSettingsRef().background_pool_size, 0, 10000, false) , move_pool(global_context.getSettingsRef().background_move_pool_size, 0, 10000, false) { @@ -14,6 +16,20 @@ BackgroundJobsExecutor::BackgroundJobsExecutor( data.getStorageID().getFullTableName() + " (dataProcessingTask)", [this]{ dataProcessingTask(); }); } +void BackgroundJobsExecutor::dataMovingTask() +try +{ + auto job = data.getDataMovingJob(); + if (job) + move_pool.scheduleOrThrowOnError(*job); + + data_moving_task->schedule(); +} +catch(...) +{ + tryLogCurrentException(__PRETTY_FUNCTION__); +} + void BackgroundJobsExecutor::dataProcessingTask() try { @@ -28,10 +44,21 @@ catch (...) tryLogCurrentException(__PRETTY_FUNCTION__); } +void BackgroundJobsExecutor::startMovingTaskIfNeeded() +{ + if (data.areBackgroundMovesNeeded() && !data_moving_task) + { + data_moving_task = global_context.getSchedulePool().createTask( + data.getStorageID().getFullTableName() + " (dataMovingTask)", [this]{ dataMovingTask(); }); + data_moving_task->activateAndSchedule(); + } +} + void BackgroundJobsExecutor::start() { if (data_processing_task) data_processing_task->activateAndSchedule(); + startMovingTaskIfNeeded(); } void BackgroundJobsExecutor::triggerDataProcessing() @@ -44,6 +71,11 @@ void BackgroundJobsExecutor::finish() { data_processing_task->deactivate(); data_processing_pool.wait(); + if (data_moving_task) + { + data_moving_task->deactivate(); + move_pool.wait(); + } } } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index aa166eb4d73..0945c4e0b59 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -12,19 +12,22 @@ class BackgroundJobsExecutor { private: MergeTreeData & data; + Context & global_context; ThreadPool data_processing_pool; ThreadPool move_pool; BackgroundSchedulePool::TaskHolder data_processing_task; - BackgroundSchedulePool::TaskHolder move_processing_task; + BackgroundSchedulePool::TaskHolder data_moving_task; void dataProcessingTask(); + void dataMovingTask(); public: BackgroundJobsExecutor( MergeTreeData & data_, Context & global_context_); + void startMovingTaskIfNeeded(); void triggerDataProcessing(); void triggerMovesProcessing(); void start(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 40f12428561..03817c70ac0 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -943,7 +943,7 @@ void IMergeTreeDataPart::makeCloneInDetached(const String & prefix, const Storag volume->getDisk()->removeIfExists(destination_path + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME); } -void IMergeTreeDataPart::makeCloneOnDiskDetached(const ReservationPtr & reservation) const +void IMergeTreeDataPart::makeCloneOnDiskDetached(const std::shared_ptr & reservation) const { assertOnDisk(); auto reserved_disk = reservation->getDisk(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 78daf6c9017..89136eaba4e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -317,7 +317,7 @@ public: virtual void makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const; /// Makes full clone of part in detached/ on another disk - void makeCloneOnDiskDetached(const ReservationPtr & reservation) const; + void makeCloneOnDiskDetached(const std::shared_ptr & reservation) const; /// Checks that .bin and .mrk files exist. /// diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a0c1de756be..4b53ecba3c4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -71,6 +71,7 @@ namespace ProfileEvents namespace CurrentMetrics { extern const Metric DelayedInserts; + extern const Metric BackgroundMovePoolTask; } @@ -3614,12 +3615,31 @@ bool MergeTreeData::selectPartsAndMove() return false; auto moving_tagger = selectPartsForMove(); - if (moving_tagger.parts_to_move.empty()) + if (moving_tagger->parts_to_move.empty()) return false; return moveParts(std::move(moving_tagger)); } +std::optional MergeTreeData::getDataMovingJob() +{ + if (parts_mover.moves_blocker.isCancelled()) + return {}; + + auto moving_tagger = selectPartsForMove(); + if (moving_tagger->parts_to_move.empty()) + return {}; + + auto job = [this, moving_tagger{std::move(moving_tagger)}] () mutable + { + moveParts(moving_tagger); + }; + + MergeTreeBackgroundJob result_job(std::move(job), CurrentMetrics::BackgroundMovePoolTask, PoolType::MOVE); + + return std::make_optional(std::move(job), CurrentMetrics::BackgroundMovePoolTask, PoolType::MOVE); +} + bool MergeTreeData::areBackgroundMovesNeeded() const { auto policy = getStoragePolicy(); @@ -3636,13 +3656,13 @@ bool MergeTreeData::movePartsToSpace(const DataPartsVector & parts, SpacePtr spa return false; auto moving_tagger = checkPartsForMove(parts, space); - if (moving_tagger.parts_to_move.empty()) + if (moving_tagger->parts_to_move.empty()) return false; - return moveParts(std::move(moving_tagger)); + return moveParts(moving_tagger); } -MergeTreeData::CurrentlyMovingPartsTagger MergeTreeData::selectPartsForMove() +MergeTreeData::CurrentlyMovingPartsTaggerPtr MergeTreeData::selectPartsForMove() { MergeTreeMovingParts parts_to_move; @@ -3665,10 +3685,10 @@ MergeTreeData::CurrentlyMovingPartsTagger MergeTreeData::selectPartsForMove() std::lock_guard moving_lock(moving_parts_mutex); parts_mover.selectPartsForMove(parts_to_move, can_move, moving_lock); - return CurrentlyMovingPartsTagger(std::move(parts_to_move), *this); + return std::make_shared(std::move(parts_to_move), *this); } -MergeTreeData::CurrentlyMovingPartsTagger MergeTreeData::checkPartsForMove(const DataPartsVector & parts, SpacePtr space) +MergeTreeData::CurrentlyMovingPartsTaggerPtr MergeTreeData::checkPartsForMove(const DataPartsVector & parts, SpacePtr space) { std::lock_guard moving_lock(moving_parts_mutex); @@ -3693,14 +3713,14 @@ MergeTreeData::CurrentlyMovingPartsTagger MergeTreeData::checkPartsForMove(const parts_to_move.emplace_back(part, std::move(reservation)); } - return CurrentlyMovingPartsTagger(std::move(parts_to_move), *this); + return std::make_shared(std::move(parts_to_move), *this); } -bool MergeTreeData::moveParts(CurrentlyMovingPartsTagger && moving_tagger) +bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagger) { - LOG_INFO(log, "Got {} parts to move.", moving_tagger.parts_to_move.size()); + LOG_INFO(log, "Got {} parts to move.", moving_tagger->parts_to_move.size()); - for (const auto & moving_part : moving_tagger.parts_to_move) + for (const auto & moving_part : moving_tagger->parts_to_move) { Stopwatch stopwatch; DataPartPtr cloned_part; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 8c5333315fc..1ebe21e98af 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -711,7 +711,9 @@ public: /// Mutex for currently_moving_parts mutable std::mutex moving_parts_mutex; - virtual std::optional getDataProcessingJob() { return {}; } + virtual std::optional getDataProcessingJob() = 0; + std::optional getDataMovingJob(); + bool areBackgroundMovesNeeded() const; protected: @@ -889,7 +891,6 @@ protected: /// Selects parts for move and moves them, used in background process bool selectPartsAndMove(); - bool areBackgroundMovesNeeded() const; private: /// RAII Wrapper for atomic work with currently moving parts @@ -901,18 +902,19 @@ private: MergeTreeData & data; CurrentlyMovingPartsTagger(MergeTreeMovingParts && moving_parts_, MergeTreeData & data_); - CurrentlyMovingPartsTagger(const CurrentlyMovingPartsTagger & other) = delete; ~CurrentlyMovingPartsTagger(); }; + using CurrentlyMovingPartsTaggerPtr = std::shared_ptr; + /// Move selected parts to corresponding disks - bool moveParts(CurrentlyMovingPartsTagger && moving_tagger); + bool moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagger); /// Select parts for move and disks for them. Used in background moving processes. - CurrentlyMovingPartsTagger selectPartsForMove(); + CurrentlyMovingPartsTaggerPtr selectPartsForMove(); /// Check selected parts for movements. Used by ALTER ... MOVE queries. - CurrentlyMovingPartsTagger checkPartsForMove(const DataPartsVector & parts, SpacePtr space); + CurrentlyMovingPartsTaggerPtr checkPartsForMove(const DataPartsVector & parts, SpacePtr space); bool canUsePolymorphicParts(const MergeTreeSettings & settings, String * out_reason = nullptr) const; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 942bac0d294..9a0db253abf 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -154,6 +154,7 @@ MergeTreeDataMergerMutator::MergeTreeDataMergerMutator(MergeTreeData & data_, si { } + UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge() const { size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); @@ -166,6 +167,7 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz { if (pool_used > pool_size) { + std::cerr << "POOLSIZE:" << pool_size << " POOL USED:" << pool_used << std::endl; throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR); } diff --git a/src/Storages/MergeTree/MergeTreePartsMover.h b/src/Storages/MergeTree/MergeTreePartsMover.h index a1afadec7fa..332a0988d10 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.h +++ b/src/Storages/MergeTree/MergeTreePartsMover.h @@ -16,7 +16,7 @@ namespace DB struct MergeTreeMoveEntry { std::shared_ptr part; - ReservationPtr reserved_space; + std::shared_ptr reserved_space; MergeTreeMoveEntry(const std::shared_ptr & part_, ReservationPtr reservation_) : part(part_), reserved_space(std::move(reservation_)) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 48caf59e7ba..b7fd7097546 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -584,8 +584,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper LOG_DEBUG(log, "Pulled {} entries to queue.", copied_entries.size()); } - if (storage.queue_task_handle) - storage.queue_task_handle->signalReadyToRun(); + storage.background_executor.triggerDataProcessing(); } return stat.version; @@ -668,8 +667,8 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C } } - if (some_active_mutations_were_killed && storage.queue_task_handle) - storage.queue_task_handle->signalReadyToRun(); + if (some_active_mutations_were_killed) + storage.background_executor.triggerDataProcessing(); if (!entries_to_load.empty()) { @@ -792,8 +791,8 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation( LOG_DEBUG(log, "Removed mutation {} from local state.", entry->znode_name); } - if (mutation_was_active && storage.queue_task_handle) - storage.queue_task_handle->signalReadyToRun(); + if (mutation_was_active) + storage.background_executor.triggerDataProcessing(); return entry; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 00ddb7a57ce..cfdd8a61e9c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -106,7 +106,6 @@ void StorageMergeTree::startup() try { background_executor.start(); - startBackgroundMovesIfNeeded(); } catch (...) { @@ -145,9 +144,6 @@ void StorageMergeTree::shutdown() background_executor.finish(); - if (moving_task_handle) - global_context.getBackgroundMovePool().removeTask(moving_task_handle); - try { /// We clear all old parts after stopping all background operations. @@ -501,18 +497,6 @@ std::optional StorageMergeTree::getIncompleteMutationsS return result; } - -void StorageMergeTree::startBackgroundMovesIfNeeded() -{ - if (areBackgroundMovesNeeded() && !moving_task_handle) - { - auto & move_pool = global_context.getBackgroundMovePool(); - moving_task_handle = move_pool.createTask([this] { return movePartsTask(); }); - move_pool.startTask(moving_task_handle); - } -} - - std::vector StorageMergeTree::getMutationsStatus() const { std::lock_guard lock(currently_processing_in_background_mutex); @@ -1530,4 +1514,9 @@ MutationCommands StorageMergeTree::getFirtsAlterMutationCommandsForPart(const Da return it->second.commands; } +void StorageMergeTree::startBackgroundMovesIfNeeded() +{ + background_executor.startMovingTaskIfNeeded(); +} + } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index c028e15416f..957b7ce56a6 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -98,6 +98,7 @@ private: MergeTreeDataSelectExecutor reader; MergeTreeDataWriter writer; MergeTreeDataMergerMutator merger_mutator; + BackgroundJobsExecutor background_executor; /// For block numbers. SimpleIncrement increment{0}; @@ -120,10 +121,6 @@ private: std::atomic shutdown_called {false}; - /// Task handler for merges, mutations and moves. - BackgroundJobsExecutor background_executor; - BackgroundProcessingPool::TaskHandle moving_task_handle; - void loadMutations(); /** Determines what parts should be merged and merges it. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ae7ad4a3518..c61f2425b17 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -197,6 +197,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , merger_mutator(*this, global_context.getSettingsRef().background_pool_size) , queue(*this) , fetcher(*this) + , background_executor(*this, global_context) , cleanup_thread(*this) , part_check_thread(*this) , restarting_thread(*this) @@ -3530,12 +3531,9 @@ void StorageReplicatedMergeTree::startup() /// between the assignment of queue_task_handle and queueTask that use the queue_task_handle. { auto lock = queue.lockQueue(); - auto & pool = global_context.getBackgroundPool(); - queue_task_handle = pool.createTask([this] { return queueTask(); }); - pool.startTask(queue_task_handle); + background_executor.start(); } - startBackgroundMovesIfNeeded(); } catch (...) { @@ -3566,14 +3564,11 @@ void StorageReplicatedMergeTree::shutdown() restarting_thread.shutdown(); - if (queue_task_handle) - global_context.getBackgroundPool().removeTask(queue_task_handle); - { /// Queue can trigger queue_task_handle itself. So we ensure that all /// queue processes finished and after that reset queue_task_handle. auto lock = queue.lockQueue(); - queue_task_handle.reset(); + background_executor.finish(); /// Cancel logs pulling after background task were cancelled. It's still /// required because we can trigger pullLogsToQueue during manual OPTIMIZE, @@ -5921,12 +5916,9 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI { auto lock = queue.lockQueue(); - if (!queue_task_handle) - return false; - + background_executor.triggerDataProcessing(); /// This is significant, because the execution of this task could be delayed at BackgroundPool. /// And we force it to be executed. - queue_task_handle->signalReadyToRun(); } Poco::Event target_size_event; @@ -6032,15 +6024,9 @@ MutationCommands StorageReplicatedMergeTree::getFirtsAlterMutationCommandsForPar return queue.getFirstAlterMutationCommandsForPart(part); } - void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded() { - if (areBackgroundMovesNeeded() && !move_parts_task_handle) - { - auto & pool = global_context.getBackgroundMovePool(); - move_parts_task_handle = pool.createTask([this] { return movePartsTask(); }); - pool.startTask(move_parts_task_handle); - } + background_executor.startMovingTaskIfNeeded(); } } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 1b65ffdbc25..1a0cabae5fa 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -27,6 +27,7 @@ #include #include #include +#include namespace DB @@ -275,15 +276,14 @@ private: int metadata_version = 0; /// Threads. + BackgroundJobsExecutor background_executor; + /// A task that keeps track of the updates in the logs of all replicas and loads them into the queue. bool queue_update_in_progress = false; BackgroundSchedulePool::TaskHolder queue_updating_task; BackgroundSchedulePool::TaskHolder mutations_updating_task; - /// A task that performs actions from the queue. - BackgroundProcessingPool::TaskHandle queue_task_handle; - /// A task which move parts to another disks/volumes /// Transparent for replication. BackgroundProcessingPool::TaskHandle move_parts_task_handle; @@ -568,7 +568,6 @@ private: MutationCommands getFirtsAlterMutationCommandsForPart(const DataPartPtr & part) const override; void startBackgroundMovesIfNeeded() override; - protected: /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. */ From 4014e0f08d0f2a12a06601264b57d2f5d3af4b84 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 15:32:35 +0300 Subject: [PATCH 021/314] Something very similar to working code --- .../MergeTree/BackgroundJobsExecutor.cpp | 100 +++++++++++++++--- .../MergeTree/BackgroundJobsExecutor.h | 7 +- .../MergeTree/MergeTreeBackgroundJob.h | 15 +-- src/Storages/MergeTree/MergeTreeData.cpp | 8 +- src/Storages/MergeTree/MergeTreeData.h | 5 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 4 +- src/Storages/StorageMergeTree.cpp | 60 +---------- src/Storages/StorageMergeTree.h | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 39 +------ src/Storages/StorageReplicatedMergeTree.h | 7 +- 10 files changed, 112 insertions(+), 137 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 081ef818dcf..f4ef89e20f9 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -1,5 +1,14 @@ #include #include +#include +#include +#include +#include + +namespace CurrentMetrics +{ + extern const Metric BackgroundPoolTask; +} namespace DB { @@ -9,11 +18,21 @@ BackgroundJobsExecutor::BackgroundJobsExecutor( Context & global_context_) : data(data_) , global_context(global_context_) - , data_processing_pool(global_context.getSettingsRef().background_pool_size, 0, 10000, false) - , move_pool(global_context.getSettingsRef().background_move_pool_size, 0, 10000, false) + , max_pool_size(global_context.getSettingsRef().background_pool_size) + , data_processing_pool(max_pool_size, 0, max_pool_size, false) + , move_pool(global_context.getSettingsRef().background_move_pool_size, 0, max_pool_size, false) + , rng(randomSeed()) { data_processing_task = global_context.getSchedulePool().createTask( data.getStorageID().getFullTableName() + " (dataProcessingTask)", [this]{ dataProcessingTask(); }); + const auto & config = global_context.getConfigRef(); + settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); + settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); + settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); + settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); + settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); + settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); + settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); } void BackgroundJobsExecutor::dataMovingTask() @@ -21,7 +40,7 @@ try { auto job = data.getDataMovingJob(); if (job) - move_pool.scheduleOrThrowOnError(*job); + move_pool.scheduleOrThrowOnError(job); data_moving_task->schedule(); } @@ -30,20 +49,77 @@ catch(...) tryLogCurrentException(__PRETTY_FUNCTION__); } -void BackgroundJobsExecutor::dataProcessingTask() -try +namespace { - auto job = data.getDataProcessingJob(); - if (job) - data_processing_pool.scheduleOrThrowOnError(*job); - data_processing_task->schedule(); -} -catch (...) +bool incrementIfLess(std::atomic & atomic_value, long max_value) { - tryLogCurrentException(__PRETTY_FUNCTION__); + auto value = atomic_value.load(std::memory_order_relaxed); + while (value < max_value) + if(atomic_value.compare_exchange_weak(value, value + 1, std::memory_order_release, std::memory_order_relaxed)) + return true; + return false; } +} + +void BackgroundJobsExecutor::dataProcessingTask() +{ + if (incrementIfLess(CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask], max_pool_size)) + { + try + { + auto job = data.getDataProcessingJob(); + if (job) + { + data_processing_pool.scheduleOrThrowOnError([this, job{std::move(job)}] () + { + try + { + job(); + CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; + errors_count = 0; + } + catch (...) + { + errors_count++; + tryLogCurrentException(__PRETTY_FUNCTION__); + CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; + } + }); + auto errors = errors_count.load(std::memory_order_relaxed); + if (errors != 0) + { + auto next_time_to_execute = 1000 * (std::min( + settings.task_sleep_seconds_when_no_work_max, + settings.task_sleep_seconds_when_no_work_min * std::pow(settings.task_sleep_seconds_when_no_work_multiplier, errors)) + + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng)); + data_processing_task->scheduleAfter(next_time_to_execute); + } + else + data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); + } + else + { + data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); + CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; + } + } + catch(...) + { + CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; + data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + else + { + /// Pool overloaded + data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); + } +} + + void BackgroundJobsExecutor::startMovingTaskIfNeeded() { if (data.areBackgroundMovesNeeded() && !data_moving_task) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 0945c4e0b59..bf1fd0c5a03 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -2,8 +2,9 @@ #include #include -#include #include +#include +#include namespace DB { @@ -13,8 +14,12 @@ class BackgroundJobsExecutor private: MergeTreeData & data; Context & global_context; + size_t max_pool_size; ThreadPool data_processing_pool; ThreadPool move_pool; + std::atomic errors_count{0}; + pcg64 rng; + BackgroundProcessingPool::PoolSettings settings; BackgroundSchedulePool::TaskHolder data_processing_task; BackgroundSchedulePool::TaskHolder data_moving_task; diff --git a/src/Storages/MergeTree/MergeTreeBackgroundJob.h b/src/Storages/MergeTree/MergeTreeBackgroundJob.h index 2f432114db1..0b1a6ff3a8d 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundJob.h +++ b/src/Storages/MergeTree/MergeTreeBackgroundJob.h @@ -17,25 +17,16 @@ enum PoolType struct MergeTreeBackgroundJob { ThreadPool::Job job; - CurrentMetrics::Metric metric; PoolType execute_in_pool; - MergeTreeBackgroundJob(ThreadPool::Job && job_, CurrentMetrics::Metric metric_, PoolType execute_in_pool_) - : job(std::move(job_)), metric(metric_), execute_in_pool(execute_in_pool_) + MergeTreeBackgroundJob(ThreadPool::Job && job_, PoolType execute_in_pool_) + : job(std::move(job_)), execute_in_pool(execute_in_pool_) {} void operator()() try { - if (metric != 0) - { - CurrentMetrics::Increment metric_increment{metric}; - job(); - } - else - { - job(); - } + job(); } catch (...) { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 4b53ecba3c4..82c01acbcf4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3621,7 +3621,7 @@ bool MergeTreeData::selectPartsAndMove() return moveParts(std::move(moving_tagger)); } -std::optional MergeTreeData::getDataMovingJob() +ThreadPool::Job MergeTreeData::getDataMovingJob() { if (parts_mover.moves_blocker.isCancelled()) return {}; @@ -3630,14 +3630,10 @@ std::optional MergeTreeData::getDataMovingJob() if (moving_tagger->parts_to_move.empty()) return {}; - auto job = [this, moving_tagger{std::move(moving_tagger)}] () mutable + return [this, moving_tagger{std::move(moving_tagger)}] () mutable { moveParts(moving_tagger); }; - - MergeTreeBackgroundJob result_job(std::move(job), CurrentMetrics::BackgroundMovePoolTask, PoolType::MOVE); - - return std::make_optional(std::move(job), CurrentMetrics::BackgroundMovePoolTask, PoolType::MOVE); } bool MergeTreeData::areBackgroundMovesNeeded() const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 1ebe21e98af..ed0e7e33d8b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -25,7 +25,6 @@ #include #include #include -#include #include #include @@ -711,8 +710,8 @@ public: /// Mutex for currently_moving_parts mutable std::mutex moving_parts_mutex; - virtual std::optional getDataProcessingJob() = 0; - std::optional getDataMovingJob(); + virtual ThreadPool::Job getDataProcessingJob() = 0; + ThreadPool::Job getDataMovingJob(); bool areBackgroundMovesNeeded() const; protected: diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 9a0db253abf..6aa3d1f3238 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -159,15 +159,15 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge() const { size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); - return getMaxSourcePartsSizeForMerge(background_pool_size, busy_threads_in_pool == 0 ? 0 : busy_threads_in_pool - 1); /// 1 is current thread + return getMaxSourcePartsSizeForMerge(background_pool_size, busy_threads_in_pool); } UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used) const { + //LOG_DEBUG(&Poco::Logger::get("DEBUG"), "POOL SIZE {}, POOL USED {}", pool_size, pool_used); if (pool_used > pool_size) { - std::cerr << "POOLSIZE:" << pool_size << " POOL USED:" << pool_used << std::endl; throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index cfdd8a61e9c..5a1ce42d831 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -210,8 +210,9 @@ void StorageMergeTree::drop() dropAllData(); } -void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) +void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder & lock_holder) { + lock_holder.release(); { /// Asks to complete merges and does not allow them to start. /// This protects against "revival" of data for a removed partition after completion of merge. @@ -902,7 +903,7 @@ bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_sn return true; } -std::optional StorageMergeTree::getDataProcessingJob() +ThreadPool::Job StorageMergeTree::getDataProcessingJob() { if (shutdown_called) return {}; @@ -919,18 +920,17 @@ std::optional StorageMergeTree::getDataProcessingJob() if (merge_entry || mutate_entry) { - auto job = [this, metadata_snapshot, merge_entry{std::move(merge_entry)}, mutate_entry{std::move(mutate_entry)}] () mutable + return [this, metadata_snapshot, merge_entry{std::move(merge_entry)}, mutate_entry{std::move(mutate_entry)}] () mutable { if (merge_entry) mergeSelectedParts(metadata_snapshot, false, *merge_entry); else if (mutate_entry) mutateSelectedPart(metadata_snapshot, *mutate_entry); }; - return std::make_optional(std::move(job), CurrentMetrics::BackgroundPoolTask, PoolType::MERGE_MUTATE); } else if (auto lock = time_after_previous_cleanup.compareAndRestartDeferred(1)) { - auto job = [this] () + return [this] () { { auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); @@ -942,60 +942,10 @@ std::optional StorageMergeTree::getDataProcessingJob() } clearOldMutations(); }; - - return std::make_optional(std::move(job), 0, PoolType::MERGE_MUTATE); } return {}; } -BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask() -{ - if (shutdown_called) - return BackgroundProcessingPoolTaskResult::ERROR; - - if (merger_mutator.merges_blocker.isCancelled()) - return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO; - - try - { - /// Clear old parts. It is unnecessary to do it more than once a second. - if (auto lock = time_after_previous_cleanup.compareAndRestartDeferred(1)) - { - { - auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); - /// All use relative_data_path which changes during rename - /// so execute under share lock. - clearOldPartsFromFilesystem(); - clearOldTemporaryDirectories(); - clearOldWriteAheadLogs(); - } - clearOldMutations(); - } - - auto metadata_snapshot = getInMemoryMetadataPtr(); - auto merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr); - ///TODO: read deduplicate option from table config - if (merge_entry && mergeSelectedParts(metadata_snapshot, false, *merge_entry)) - return BackgroundProcessingPoolTaskResult::SUCCESS; - - auto mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr); - if (mutate_entry && mutateSelectedPart(metadata_snapshot, *mutate_entry)) - return BackgroundProcessingPoolTaskResult::SUCCESS; - - return BackgroundProcessingPoolTaskResult::ERROR; - } - catch (const Exception & e) - { - if (e.code() == ErrorCodes::ABORTED) - { - LOG_INFO(log, e.message()); - return BackgroundProcessingPoolTaskResult::ERROR; - } - - throw; - } -} - Int64 StorageMergeTree::getCurrentMutationVersion( const DataPartPtr & part, std::unique_lock & /* currently_processing_in_background_mutex_lock */) const diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 957b7ce56a6..0c2b633fa70 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -88,7 +88,7 @@ public: CheckResults checkData(const ASTPtr & query, const Context & context) override; - std::optional getDataProcessingJob() override; + ThreadPool::Job getDataProcessingJob() override; private: /// Mutex and condvar for synchronous mutations wait @@ -156,8 +156,6 @@ private: std::optional selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason); bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry); - BackgroundProcessingPoolTaskResult mergeMutateTask(); - Int64 getCurrentMutationVersion( const DataPartPtr & part, std::unique_lock & /* currently_processing_in_background_mutex_lock */) const; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c61f2425b17..1d1b72018e6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2601,7 +2601,7 @@ bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::Sel } -std::optional StorageReplicatedMergeTree::getDataProcessingJob() +ThreadPool::Job StorageReplicatedMergeTree::getDataProcessingJob() { /// If replication queue is stopped exit immediately as we successfully executed the task if (queue.actions_blocker.isCancelled()) @@ -2615,43 +2615,12 @@ std::optional StorageReplicatedMergeTree::getDataProcess if (!entry) return {}; - auto job = [this, selected_entry{std::move(selected_entry)}] () mutable + return [this, selected_entry{std::move(selected_entry)}] () mutable { processQueueEntry(selected_entry); }; - - return std::make_optional(std::move(job), CurrentMetrics::BackgroundPoolTask, PoolType::MERGE_MUTATE); } -BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask() -{ - /// If replication queue is stopped exit immediately as we successfully executed the task - if (queue.actions_blocker.isCancelled()) - { - std::this_thread::sleep_for(std::chrono::milliseconds(5)); - return BackgroundProcessingPoolTaskResult::SUCCESS; - } - - /// This object will mark the element of the queue as running. - ReplicatedMergeTreeQueue::SelectedEntry selected_entry = selectQueueEntry(); - - LogEntryPtr & entry = selected_entry.first; - - if (!entry) - return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO; - - time_t prev_attempt_time = entry->last_attempt_time; - - bool res = processQueueEntry(selected_entry); - - /// We will go to sleep if the processing fails and if we have already processed this record recently. - bool need_sleep = !res && (entry->last_attempt_time - prev_attempt_time < 10); - - /// If there was no exception, you do not need to sleep. - return need_sleep ? BackgroundProcessingPoolTaskResult::ERROR : BackgroundProcessingPoolTaskResult::SUCCESS; -} - - bool StorageReplicatedMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & part) const { return queue.isVirtualPart(part); @@ -3576,10 +3545,6 @@ void StorageReplicatedMergeTree::shutdown() queue.pull_log_blocker.cancelForever(); } - if (move_parts_task_handle) - global_context.getBackgroundMovePool().removeTask(move_parts_task_handle); - move_parts_task_handle.reset(); - if (data_parts_exchange_endpoint) { global_context.getInterserverIOHandler().removeEndpointIfExists(data_parts_exchange_endpoint->getId(replica_path)); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 1a0cabae5fa..6c30d966afd 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -196,7 +196,7 @@ public: */ static void dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica, Poco::Logger * logger); - std::optional getDataProcessingJob() override; + ThreadPool::Job getDataProcessingJob() override; private: @@ -423,15 +423,10 @@ private: bool processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntry & entry); - /** Performs actions from the queue. - */ - BackgroundProcessingPoolTaskResult queueTask(); - /// Perform moves of parts to another disks. /// Local operation, doesn't interact with replicationg queue. BackgroundProcessingPoolTaskResult movePartsTask(); - /// 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 0b14a31ba9068c6bf270ef285b4f2b3b9816bca5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 15:44:10 +0300 Subject: [PATCH 022/314] Get rid of background processing pool --- src/Interpreters/Context.cpp | 44 ------------------- src/Interpreters/Context.h | 3 -- .../MergeTree/BackgroundJobsExecutor.h | 31 ++++++++++++- src/Storages/StorageMergeTree.cpp | 17 ------- src/Storages/StorageMergeTree.h | 3 -- src/Storages/StorageReplicatedMergeTree.cpp | 17 ------- src/Storages/StorageReplicatedMergeTree.h | 9 ---- 7 files changed, 29 insertions(+), 95 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index e7614a66761..e98a8d59155 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -331,8 +330,6 @@ struct ContextShared ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections. InterserverIOHandler interserver_io_handler; /// Handler for interserver communication. std::optional buffer_flush_schedule_pool; /// A thread pool that can do background flush for Buffer tables. - std::optional background_pool; /// The thread pool for the background work performed by the tables. - std::optional background_move_pool; /// The thread pool for the background moves performed by the tables. std::optional schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables) std::optional distributed_schedule_pool; /// A thread pool that can run different jobs in background (used for distributed sends) std::optional message_broker_schedule_pool; /// A thread pool that can run different jobs in background (used in kafka streaming) @@ -433,8 +430,6 @@ struct ContextShared external_dictionaries_loader.reset(); external_models_loader.reset(); buffer_flush_schedule_pool.reset(); - background_pool.reset(); - background_move_pool.reset(); schedule_pool.reset(); distributed_schedule_pool.reset(); ddl_worker.reset(); @@ -1369,45 +1364,6 @@ void Context::dropCaches() const shared->mark_cache->reset(); } -BackgroundProcessingPool & Context::getBackgroundPool() -{ - auto lock = getLock(); - if (!shared->background_pool) - { - BackgroundProcessingPool::PoolSettings pool_settings; - const auto & config = getConfigRef(); - pool_settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); - pool_settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); - pool_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - pool_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); - pool_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); - pool_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); - pool_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); - shared->background_pool.emplace(settings.background_pool_size, pool_settings); - } - return *shared->background_pool; -} - -BackgroundProcessingPool & Context::getBackgroundMovePool() -{ - auto lock = getLock(); - if (!shared->background_move_pool) - { - BackgroundProcessingPool::PoolSettings pool_settings; - const auto & config = getConfigRef(); - pool_settings.thread_sleep_seconds = config.getDouble("background_move_processing_pool_thread_sleep_seconds", 10); - pool_settings.thread_sleep_seconds_random_part = config.getDouble("background_move_processing_pool_thread_sleep_seconds_random_part", 1.0); - pool_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_move_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - pool_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 10); - pool_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 600); - pool_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); - pool_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); - pool_settings.tasks_metric = CurrentMetrics::BackgroundMovePoolTask; - shared->background_move_pool.emplace(settings.background_move_pool_size, pool_settings, "BackgroundMovePool", "BgMoveProcPool"); - } - return *shared->background_move_pool; -} - BackgroundSchedulePool & Context::getBufferFlushSchedulePool() { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index bd5e17fe2e4..155d8fbcd73 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -62,7 +62,6 @@ class EmbeddedDictionaries; class ExternalDictionariesLoader; class ExternalModelsLoader; class InterserverIOHandler; -class BackgroundProcessingPool; class BackgroundSchedulePool; class MergeList; class Cluster; @@ -508,8 +507,6 @@ public: void dropCaches() const; BackgroundSchedulePool & getBufferFlushSchedulePool(); - BackgroundProcessingPool & getBackgroundPool(); - BackgroundProcessingPool & getBackgroundMovePool(); BackgroundSchedulePool & getSchedulePool(); BackgroundSchedulePool & getMessageBrokerSchedulePool(); BackgroundSchedulePool & getDistributedSchedulePool(); diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index bf1fd0c5a03..b7bd63f7169 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -3,11 +3,38 @@ #include #include #include -#include #include +namespace CurrentMetrics +{ + extern const Metric BackgroundPoolTask; +} + namespace DB { +enum PoolType +{ + MERGE_MUTATING, + MOVING, + +} + +struct PoolSettings +{ + double thread_sleep_seconds = 10; + double thread_sleep_seconds_random_part = 1.0; + double thread_sleep_seconds_if_nothing_to_do = 0.1; + + /// For exponential backoff. + double task_sleep_seconds_when_no_work_min = 10; + double task_sleep_seconds_when_no_work_max = 600; + double task_sleep_seconds_when_no_work_multiplier = 1.1; + double task_sleep_seconds_when_no_work_random_part = 1.0; + + CurrentMetrics::Metric tasks_metric = CurrentMetrics::BackgroundPoolTask; + + PoolSettings() noexcept {} +}; class BackgroundJobsExecutor { @@ -19,7 +46,7 @@ private: ThreadPool move_pool; std::atomic errors_count{0}; pcg64 rng; - BackgroundProcessingPool::PoolSettings settings; + PoolSettings settings; BackgroundSchedulePool::TaskHolder data_processing_task; BackgroundSchedulePool::TaskHolder data_moving_task; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 5a1ce42d831..87dfbd4d879 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -762,23 +762,6 @@ bool StorageMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & p return currently_merging_mutating_parts.count(part); } - -BackgroundProcessingPoolTaskResult StorageMergeTree::movePartsTask() -{ - try - { - if (!selectPartsAndMove()) - return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO; - - return BackgroundProcessingPoolTaskResult::SUCCESS; - } - catch (...) - { - tryLogCurrentException(log); - return BackgroundProcessingPoolTaskResult::ERROR; - } -} - std::optional StorageMergeTree::selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String */* disable_reason */) { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 0c2b633fa70..b1946e48d4f 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -13,7 +13,6 @@ #include #include #include -#include #include #include #include @@ -131,8 +130,6 @@ private: ActionLock stopMergesAndWait(); - BackgroundProcessingPoolTaskResult movePartsTask(); - /// Allocate block number for new mutation, write mutation to disk /// and into in-memory structures. Wake up merge-mutation task. Int64 startMutation(const MutationCommands & commands, String & mutation_file_name); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1d1b72018e6..453956ffa8b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2626,23 +2626,6 @@ bool StorageReplicatedMergeTree::partIsAssignedToBackgroundOperation(const DataP return queue.isVirtualPart(part); } -BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::movePartsTask() -{ - try - { - if (!selectPartsAndMove()) - return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO; - - return BackgroundProcessingPoolTaskResult::SUCCESS; - } - catch (...) - { - tryLogCurrentException(log); - return BackgroundProcessingPoolTaskResult::ERROR; - } -} - - void StorageReplicatedMergeTree::mergeSelectingTask() { if (!is_leader) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 6c30d966afd..4ce3c0ad3c1 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -16,7 +16,6 @@ #include #include #include -#include #include #include #include @@ -284,10 +283,6 @@ private: BackgroundSchedulePool::TaskHolder mutations_updating_task; - /// A task which move parts to another disks/volumes - /// Transparent for replication. - BackgroundProcessingPool::TaskHandle move_parts_task_handle; - /// A task that selects parts to merge. BackgroundSchedulePool::TaskHolder merge_selecting_task; /// It is acquired for each iteration of the selection of parts to merge or each OPTIMIZE query. @@ -423,10 +418,6 @@ private: bool processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntry & entry); - /// Perform moves of parts to another disks. - /// Local operation, doesn't interact with replicationg queue. - BackgroundProcessingPoolTaskResult movePartsTask(); - /// 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 483893cdd40df6e1a47ace63842cd8d2f1d4491d Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 17:56:42 +0300 Subject: [PATCH 023/314] Slightly refactored code --- .../MergeTree/BackgroundJobsExecutor.cpp | 344 +++++++++++++----- .../MergeTree/BackgroundJobsExecutor.h | 99 +++-- src/Storages/StorageMergeTree.cpp | 7 +- src/Storages/StorageMergeTree.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 6 +- src/Storages/StorageReplicatedMergeTree.h | 1 + 6 files changed, 336 insertions(+), 122 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index f4ef89e20f9..16bb5b3c4f0 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -5,48 +5,31 @@ #include #include -namespace CurrentMetrics -{ - extern const Metric BackgroundPoolTask; -} - namespace DB { -BackgroundJobsExecutor::BackgroundJobsExecutor( - MergeTreeData & data_, - Context & global_context_) +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +IBackgroundJobExecutor::IBackgroundJobExecutor( + MergeTreeData & data_, + Context & global_context_, + const String & task_name_, + const TaskSleepSettings & sleep_settings_, + const std::vector & pools_configs_) : data(data_) , global_context(global_context_) - , max_pool_size(global_context.getSettingsRef().background_pool_size) - , data_processing_pool(max_pool_size, 0, max_pool_size, false) - , move_pool(global_context.getSettingsRef().background_move_pool_size, 0, max_pool_size, false) + , task_name(task_name_) + , sleep_settings(sleep_settings_) , rng(randomSeed()) { - data_processing_task = global_context.getSchedulePool().createTask( - data.getStorageID().getFullTableName() + " (dataProcessingTask)", [this]{ dataProcessingTask(); }); - const auto & config = global_context.getConfigRef(); - settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); - settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); - settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); - settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); - settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); - settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); -} - -void BackgroundJobsExecutor::dataMovingTask() -try -{ - auto job = data.getDataMovingJob(); - if (job) - move_pool.scheduleOrThrowOnError(job); - - data_moving_task->schedule(); -} -catch(...) -{ - tryLogCurrentException(__PRETTY_FUNCTION__); + for (const auto & pool_config : pools_configs_) + { + pools.try_emplace(pool_config.pool_type, pool_config.max_pool_size, 0, pool_config.max_pool_size, false); + pools_configs.emplace(pool_config.pool_type, pool_config); + } } namespace @@ -63,95 +46,270 @@ bool incrementIfLess(std::atomic & atomic_value, long max_value) } -void BackgroundJobsExecutor::dataProcessingTask() + +void IBackgroundJobExecutor::scheduleTask(bool nothing_to_do) { - if (incrementIfLess(CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask], max_pool_size)) + auto errors = errors_count.load(std::memory_order_relaxed); + size_t next_time_to_execute = 0; + if (errors != 0) + next_time_to_execute += 1000 * (std::min( + sleep_settings.task_sleep_seconds_when_no_work_max, + sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, errors)) + + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + else if (nothing_to_do) + next_time_to_execute += 1000 * (sleep_settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + else + next_time_to_execute = 1000 * std::uniform_real_distribution(0, sleep_settings.thread_sleep_seconds_random_part)(rng); + + scheduling_task->scheduleAfter(next_time_to_execute); +} + +void IBackgroundJobExecutor::jobExecutingTask() +try +{ + auto job_and_pool = getBackgroundJob(); + if (job_and_pool) { - try + auto & pool_config = pools_configs[job_and_pool->pool_type]; + /// If corresponding pool is not full, otherwise try next time + if (incrementIfLess(CurrentMetrics::values[pool_config.tasks_metric], pool_config.max_pool_size)) { - auto job = data.getDataProcessingJob(); - if (job) + try /// this try required because we have to manually decrement metric { - data_processing_pool.scheduleOrThrowOnError([this, job{std::move(job)}] () + pools[job_and_pool->pool_type].scheduleOrThrowOnError([this, pool_config, job{std::move(job_and_pool->job)}] () { - try + try /// We don't want exceptions in background pool { job(); - CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; + CurrentMetrics::values[pool_config.tasks_metric]--; errors_count = 0; } catch (...) { errors_count++; tryLogCurrentException(__PRETTY_FUNCTION__); - CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; + CurrentMetrics::values[pool_config.tasks_metric]--; } }); - auto errors = errors_count.load(std::memory_order_relaxed); - if (errors != 0) - { - auto next_time_to_execute = 1000 * (std::min( - settings.task_sleep_seconds_when_no_work_max, - settings.task_sleep_seconds_when_no_work_min * std::pow(settings.task_sleep_seconds_when_no_work_multiplier, errors)) - + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng)); - data_processing_task->scheduleAfter(next_time_to_execute); - } - else - data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); } - else + catch (...) { - data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); - CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; + tryLogCurrentException(__PRETTY_FUNCTION__); + CurrentMetrics::values[pool_config.tasks_metric]--; } } - catch(...) - { - CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; - data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); - tryLogCurrentException(__PRETTY_FUNCTION__); - } + scheduleTask(false); } - else + else /// Nothing to do, no jobs { - /// Pool overloaded - data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); + scheduleTask(true); } } - - -void BackgroundJobsExecutor::startMovingTaskIfNeeded() +catch (...) /// Exception while we looking for task { - if (data.areBackgroundMovesNeeded() && !data_moving_task) + tryLogCurrentException(__PRETTY_FUNCTION__); + scheduleTask(true); +} + +void IBackgroundJobExecutor::start() +{ + if (!scheduling_task) { - data_moving_task = global_context.getSchedulePool().createTask( - data.getStorageID().getFullTableName() + " (dataMovingTask)", [this]{ dataMovingTask(); }); - data_moving_task->activateAndSchedule(); + scheduling_task = global_context.getSchedulePool().createTask( + data.getStorageID().getFullTableName() + task_name, [this]{ jobExecutingTask(); }); } + + scheduling_task->activateAndSchedule(); } -void BackgroundJobsExecutor::start() +void IBackgroundJobExecutor::finish() { - if (data_processing_task) - data_processing_task->activateAndSchedule(); - startMovingTaskIfNeeded(); -} - -void BackgroundJobsExecutor::triggerDataProcessing() -{ - if (data_processing_task) - data_processing_task->schedule(); -} - -void BackgroundJobsExecutor::finish() -{ - data_processing_task->deactivate(); - data_processing_pool.wait(); - if (data_moving_task) + if (scheduling_task) { - data_moving_task->deactivate(); - move_pool.wait(); + scheduling_task->deactivate(); + for (auto & [pool_type, pool] : pools) + pool.wait(); } } +void IBackgroundJobExecutor::triggerDataProcessing() +{ + if (scheduling_task) + scheduling_task->schedule(); +} + + IBackgroundJobExecutor::~IBackgroundJobExecutor() +{ + finish(); +} + +BackgroundJobsExecutor::BackgroundJobsExecutor( + MergeTreeData & data_, + Context & global_context_) + : IBackgroundJobExecutor( + data_, + global_context_, + "(dataProcessingTask)", + TaskSleepSettings{}, + {PoolConfig{PoolType::MERGE_MUTATE, global_context_.getSettingsRef().background_pool_size, CurrentMetrics::BackgroundPoolTask}}) +{ +} + +std::optional BackgroundJobsExecutor::getBackgroundJob() +{ + auto job = data.getDataProcessingJob(); + if (job) + return JobAndPool{job, PoolType::MERGE_MUTATE}; + return {}; +} + +BackgroundMovesExecutor::BackgroundMovesExecutor( + MergeTreeData & data_, + Context & global_context_) + : IBackgroundJobExecutor( + data_, + global_context_, + "(dataMovingTask)", + TaskSleepSettings{}, + {PoolConfig{PoolType::MOVE, global_context_.getSettingsRef().background_move_pool_size, CurrentMetrics::BackgroundMovePoolTask}}) +{ +} + +std::optional BackgroundMovesExecutor::getBackgroundJob() +{ + auto job = data.getDataMovingJob(); + if (job) + return JobAndPool{job, PoolType::MOVE}; + return {}; +} + +//BackgroundJobsExecutor::BackgroundJobsExecutor( +// MergeTreeData & data_, +// Context & global_context_) +// : data(data_) +// , global_context(global_context_) +// , max_pool_size(global_context.getSettingsRef().background_pool_size) +// , data_processing_pool(max_pool_size, 0, max_pool_size, false) +// , move_pool(global_context.getSettingsRef().background_move_pool_size, 0, max_pool_size, false) +// , rng(randomSeed()) +//{ +// data_processing_task = global_context.getSchedulePool().createTask( +// data.getStorageID().getFullTableName() + " (dataProcessingTask)", [this]{ dataProcessingTask(); }); +// const auto & config = global_context.getConfigRef(); +// settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); +// settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); +// settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); +// settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); +// settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); +// settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); +// settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); +//} +// +//void BackgroundJobsExecutor::dataMovingTask() +//try +//{ +// auto job = data.getDataMovingJob(); +// if (job) +// move_pool.scheduleOrThrowOnError(job); +// +// data_moving_task->schedule(); +//} +//catch(...) +//{ +// tryLogCurrentException(__PRETTY_FUNCTION__); +//} +// +// +// +//void BackgroundJobsExecutor::dataProcessingTask() +//{ +// if (incrementIfLess(CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask], max_pool_size)) +// { +// try +// { +// auto job = data.getDataProcessingJob(); +// if (job) +// { +// data_processing_pool.scheduleOrThrowOnError([this, job{std::move(job)}] () +// { +// try +// { +// job(); +// CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; +// errors_count = 0; +// } +// catch (...) +// { +// errors_count++; +// tryLogCurrentException(__PRETTY_FUNCTION__); +// CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; +// } +// }); +// auto errors = errors_count.load(std::memory_order_relaxed); +// if (errors != 0) +// { +// auto next_time_to_execute = 1000 * (std::min( +// settings.task_sleep_seconds_when_no_work_max, +// settings.task_sleep_seconds_when_no_work_min * std::pow(settings.task_sleep_seconds_when_no_work_multiplier, errors)) +// + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng)); +// data_processing_task->scheduleAfter(next_time_to_execute); +// } +// else +// data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); +// } +// else +// { +// data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); +// CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; +// } +// } +// catch(...) +// { +// CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; +// data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); +// tryLogCurrentException(__PRETTY_FUNCTION__); +// } +// } +// else +// { +// /// Pool overloaded +// data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); +// } +//} +// +// +//void BackgroundJobsExecutor::startMovingTaskIfNeeded() +//{ +// if (data.areBackgroundMovesNeeded() && !data_moving_task) +// { +// data_moving_task = global_context.getSchedulePool().createTask( +// data.getStorageID().getFullTableName() + " (dataMovingTask)", [this]{ dataMovingTask(); }); +// data_moving_task->activateAndSchedule(); +// } +//} +// +//void BackgroundJobsExecutor::start() +//{ +// if (data_processing_task) +// data_processing_task->activateAndSchedule(); +// startMovingTaskIfNeeded(); +//} +// +//void BackgroundJobsExecutor::triggerDataProcessing() +//{ +// if (data_processing_task) +// data_processing_task->schedule(); +//} +// +//void BackgroundJobsExecutor::finish() +//{ +// data_processing_task->deactivate(); +// data_processing_pool.wait(); +// if (data_moving_task) +// { +// data_moving_task->deactivate(); +// move_pool.wait(); +// } +//} + } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index b7bd63f7169..4d01198f681 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -8,18 +8,14 @@ namespace CurrentMetrics { extern const Metric BackgroundPoolTask; + extern const Metric BackgroundMovePoolTask; } namespace DB { -enum PoolType -{ - MERGE_MUTATING, - MOVING, -} -struct PoolSettings +struct TaskSleepSettings { double thread_sleep_seconds = 10; double thread_sleep_seconds_random_part = 1.0; @@ -29,41 +25,90 @@ struct PoolSettings double task_sleep_seconds_when_no_work_min = 10; double task_sleep_seconds_when_no_work_max = 600; double task_sleep_seconds_when_no_work_multiplier = 1.1; + double task_sleep_seconds_when_no_work_random_part = 1.0; - - CurrentMetrics::Metric tasks_metric = CurrentMetrics::BackgroundPoolTask; - - PoolSettings() noexcept {} }; -class BackgroundJobsExecutor +enum PoolType { -private: + MERGE_MUTATE, + FETCH, + MOVE, + LOW_PRIORITY, +}; + +struct PoolConfig +{ + PoolType pool_type; + size_t max_pool_size; + CurrentMetrics::Metric tasks_metric; +}; + +struct JobAndPool +{ + ThreadPool::Job job; + PoolType pool_type; +}; + +class IBackgroundJobExecutor +{ +protected: MergeTreeData & data; Context & global_context; - size_t max_pool_size; - ThreadPool data_processing_pool; - ThreadPool move_pool; - std::atomic errors_count{0}; +private: + String task_name; + TaskSleepSettings sleep_settings; pcg64 rng; - PoolSettings settings; - BackgroundSchedulePool::TaskHolder data_processing_task; - BackgroundSchedulePool::TaskHolder data_moving_task; + std::atomic errors_count{0}; - void dataProcessingTask(); - void dataMovingTask(); + std::unordered_map pools; + std::unordered_map pools_configs; + BackgroundSchedulePool::TaskHolder scheduling_task; + +public: + IBackgroundJobExecutor( + MergeTreeData & data_, + Context & global_context_, + const String & task_name_, + const TaskSleepSettings & sleep_settings_, + const std::vector & pools_configs_); + + void start(); + void triggerDataProcessing(); + void finish(); + + virtual ~IBackgroundJobExecutor(); + +protected: + virtual std::optional getBackgroundJob() = 0; +private: + void jobExecutingTask(); + void scheduleTask(bool nothing_to_do); +}; + + +class BackgroundJobsExecutor final : public IBackgroundJobExecutor +{ public: BackgroundJobsExecutor( MergeTreeData & data_, Context & global_context_); - void startMovingTaskIfNeeded(); - void triggerDataProcessing(); - void triggerMovesProcessing(); - void start(); - void finish(); +protected: + std::optional getBackgroundJob() override; }; - + +class BackgroundMovesExecutor final : public IBackgroundJobExecutor +{ +public: + BackgroundMovesExecutor( + MergeTreeData & data_, + Context & global_context_); + +protected: + std::optional getBackgroundJob() override; +}; + } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 87dfbd4d879..c4f8436678d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -79,6 +79,8 @@ StorageMergeTree::StorageMergeTree( , writer(*this) , merger_mutator(*this, global_context.getSettingsRef().background_pool_size) , background_executor(*this, global_context) + , background_moves_executor(*this, global_context) + { loadDataParts(has_force_restore_data_flag); @@ -106,6 +108,8 @@ void StorageMergeTree::startup() try { background_executor.start(); + if (areBackgroundMovesNeeded()) + background_moves_executor.start(); } catch (...) { @@ -143,6 +147,7 @@ void StorageMergeTree::shutdown() parts_mover.moves_blocker.cancelForever(); background_executor.finish(); + background_moves_executor.finish(); try { @@ -1449,7 +1454,7 @@ MutationCommands StorageMergeTree::getFirtsAlterMutationCommandsForPart(const Da void StorageMergeTree::startBackgroundMovesIfNeeded() { - background_executor.startMovingTaskIfNeeded(); + background_executor.start(); } } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index b1946e48d4f..f7473872189 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -98,6 +98,7 @@ private: MergeTreeDataWriter writer; MergeTreeDataMergerMutator merger_mutator; BackgroundJobsExecutor background_executor; + BackgroundMovesExecutor background_moves_executor; /// For block numbers. SimpleIncrement increment{0}; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 453956ffa8b..bc55b87fee7 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -198,6 +198,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , queue(*this) , fetcher(*this) , background_executor(*this, global_context) + , background_moves_executor(*this, global_context) , cleanup_thread(*this) , part_check_thread(*this) , restarting_thread(*this) @@ -3485,6 +3486,8 @@ void StorageReplicatedMergeTree::startup() auto lock = queue.lockQueue(); background_executor.start(); } + if (areBackgroundMovesNeeded()) + background_moves_executor.start(); } catch (...) @@ -3527,6 +3530,7 @@ void StorageReplicatedMergeTree::shutdown() /// MUTATE, etc. query. queue.pull_log_blocker.cancelForever(); } + background_moves_executor.finish(); if (data_parts_exchange_endpoint) { @@ -5974,7 +5978,7 @@ MutationCommands StorageReplicatedMergeTree::getFirtsAlterMutationCommandsForPar void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded() { - background_executor.startMovingTaskIfNeeded(); + background_moves_executor.start(); } } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 4ce3c0ad3c1..ba01ca9d3af 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -276,6 +276,7 @@ private: /// Threads. BackgroundJobsExecutor background_executor; + BackgroundMovesExecutor background_moves_executor; /// A task that keeps track of the updates in the logs of all replicas and loads them into the queue. bool queue_update_in_progress = false; From 6a4911dc576dfffa82043f2c43da1db20c20fe7f Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 18:05:14 +0300 Subject: [PATCH 024/314] Remove some garbage --- .../MergeTree/BackgroundJobsExecutor.cpp | 129 ------------------ 1 file changed, 129 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 16bb5b3c4f0..c51028862e6 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -183,133 +183,4 @@ std::optional BackgroundMovesExecutor::getBackgroundJob() return {}; } -//BackgroundJobsExecutor::BackgroundJobsExecutor( -// MergeTreeData & data_, -// Context & global_context_) -// : data(data_) -// , global_context(global_context_) -// , max_pool_size(global_context.getSettingsRef().background_pool_size) -// , data_processing_pool(max_pool_size, 0, max_pool_size, false) -// , move_pool(global_context.getSettingsRef().background_move_pool_size, 0, max_pool_size, false) -// , rng(randomSeed()) -//{ -// data_processing_task = global_context.getSchedulePool().createTask( -// data.getStorageID().getFullTableName() + " (dataProcessingTask)", [this]{ dataProcessingTask(); }); -// const auto & config = global_context.getConfigRef(); -// settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); -// settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); -// settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); -// settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); -// settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); -// settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); -// settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); -//} -// -//void BackgroundJobsExecutor::dataMovingTask() -//try -//{ -// auto job = data.getDataMovingJob(); -// if (job) -// move_pool.scheduleOrThrowOnError(job); -// -// data_moving_task->schedule(); -//} -//catch(...) -//{ -// tryLogCurrentException(__PRETTY_FUNCTION__); -//} -// -// -// -//void BackgroundJobsExecutor::dataProcessingTask() -//{ -// if (incrementIfLess(CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask], max_pool_size)) -// { -// try -// { -// auto job = data.getDataProcessingJob(); -// if (job) -// { -// data_processing_pool.scheduleOrThrowOnError([this, job{std::move(job)}] () -// { -// try -// { -// job(); -// CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; -// errors_count = 0; -// } -// catch (...) -// { -// errors_count++; -// tryLogCurrentException(__PRETTY_FUNCTION__); -// CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; -// } -// }); -// auto errors = errors_count.load(std::memory_order_relaxed); -// if (errors != 0) -// { -// auto next_time_to_execute = 1000 * (std::min( -// settings.task_sleep_seconds_when_no_work_max, -// settings.task_sleep_seconds_when_no_work_min * std::pow(settings.task_sleep_seconds_when_no_work_multiplier, errors)) -// + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng)); -// data_processing_task->scheduleAfter(next_time_to_execute); -// } -// else -// data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); -// } -// else -// { -// data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); -// CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; -// } -// } -// catch(...) -// { -// CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; -// data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); -// tryLogCurrentException(__PRETTY_FUNCTION__); -// } -// } -// else -// { -// /// Pool overloaded -// data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); -// } -//} -// -// -//void BackgroundJobsExecutor::startMovingTaskIfNeeded() -//{ -// if (data.areBackgroundMovesNeeded() && !data_moving_task) -// { -// data_moving_task = global_context.getSchedulePool().createTask( -// data.getStorageID().getFullTableName() + " (dataMovingTask)", [this]{ dataMovingTask(); }); -// data_moving_task->activateAndSchedule(); -// } -//} -// -//void BackgroundJobsExecutor::start() -//{ -// if (data_processing_task) -// data_processing_task->activateAndSchedule(); -// startMovingTaskIfNeeded(); -//} -// -//void BackgroundJobsExecutor::triggerDataProcessing() -//{ -// if (data_processing_task) -// data_processing_task->schedule(); -//} -// -//void BackgroundJobsExecutor::finish() -//{ -// data_processing_task->deactivate(); -// data_processing_pool.wait(); -// if (data_moving_task) -// { -// data_moving_task->deactivate(); -// move_pool.wait(); -// } -//} - } From fc25b6cd724f563943e026e314cf2495c196b56e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 18:26:00 +0300 Subject: [PATCH 025/314] Remove background processing pool --- .../MergeTree/BackgroundProcessingPool.cpp | 240 ------------------ .../MergeTree/BackgroundProcessingPool.h | 163 ------------ 2 files changed, 403 deletions(-) delete mode 100644 src/Storages/MergeTree/BackgroundProcessingPool.cpp delete mode 100644 src/Storages/MergeTree/BackgroundProcessingPool.h diff --git a/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/src/Storages/MergeTree/BackgroundProcessingPool.cpp deleted file mode 100644 index 3dd1dda0fbf..00000000000 --- a/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ /dev/null @@ -1,240 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - - -namespace DB -{ - -void BackgroundProcessingPoolTaskInfo::signalReadyToRun() -{ - Poco::Timestamp current_time; - { - std::unique_lock lock(pool.tasks_mutex); - - /// This check ensures that the iterator is valid. Must be performed under the same mutex as invalidation. - if (removed) - return; - - /// If this task did nothing the previous time and still should sleep, then reschedule to cancel the sleep. - const auto & scheduled_time = iterator->first; - if (scheduled_time > current_time) - pool.rescheduleTask(iterator, current_time); - - /// Note that if all threads are currently busy doing their work, this call will not wakeup any thread. - pool.wake_event.notify_one(); - } -} - - -BackgroundProcessingPool::BackgroundProcessingPool(int size_, - const PoolSettings & pool_settings, - const char * log_name, - const char * thread_name_) - : size(size_) - , thread_name(thread_name_) - , settings(pool_settings) -{ - logger = &Poco::Logger::get(log_name); - LOG_INFO(logger, "Create {} with {} threads", log_name, size); - - threads.resize(size); - for (auto & thread : threads) - thread = ThreadFromGlobalPool([this] { workLoopFunc(); }); -} - - -BackgroundProcessingPool::TaskHandle BackgroundProcessingPool::createTask(const Task & task) -{ - return std::make_shared(*this, task); -} - -void BackgroundProcessingPool::startTask(const TaskHandle & task, bool allow_execute_in_parallel) -{ - Poco::Timestamp current_time; - - task->allow_execute_in_parallel = allow_execute_in_parallel; - - { - std::unique_lock lock(tasks_mutex); - task->iterator = tasks.emplace(current_time, task); - - wake_event.notify_all(); - } - -} - -BackgroundProcessingPool::TaskHandle BackgroundProcessingPool::addTask(const Task & task) -{ - TaskHandle res = createTask(task); - startTask(res); - return res; -} - -void BackgroundProcessingPool::removeTask(const TaskHandle & task) -{ - if (task->removed.exchange(true)) - return; - - /// Wait for all executions of this task. - { - std::unique_lock wlock(task->rwlock); - } - - { - std::unique_lock lock(tasks_mutex); - tasks.erase(task->iterator); - /// Note that the task may be still accessible through TaskHandle (shared_ptr). - } -} - -BackgroundProcessingPool::~BackgroundProcessingPool() -{ - try - { - { - std::lock_guard lock(tasks_mutex); - shutdown = true; - wake_event.notify_all(); - } - - for (auto & thread : threads) - thread.join(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} - - -void BackgroundProcessingPool::workLoopFunc() -{ - setThreadName(thread_name); - - { - std::lock_guard lock(tasks_mutex); - - if (thread_group) - { - /// Put all threads to one thread pool - CurrentThread::attachTo(thread_group); - } - else - { - CurrentThread::initializeQuery(); - thread_group = CurrentThread::getGroup(); - } - } - - SCOPE_EXIT({ CurrentThread::detachQueryIfNotDetached(); }); - - pcg64 rng(randomSeed()); - std::this_thread::sleep_for(std::chrono::duration(std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); - - Poco::Timestamp scheduled_task_start_time; - - while (true) - { - TaskResult task_result = TaskResult::ERROR; - TaskHandle task; - - { - std::unique_lock lock(tasks_mutex); - - while (!task && !shutdown) - { - for (const auto & [time, handle] : tasks) - { - if (!handle->removed - && (handle->allow_execute_in_parallel || handle->concurrent_executors == 0)) - { - task = handle; - scheduled_task_start_time = time; - ++task->concurrent_executors; - break; - } - } - - if (task) - { - Poco::Timestamp current_time; - - if (scheduled_task_start_time <= current_time) - continue; - - wake_event.wait_for(lock, - std::chrono::microseconds(scheduled_task_start_time - current_time - + std::uniform_int_distribution(0, settings.thread_sleep_seconds_random_part * 1000000)(rng))); - } - else - { - wake_event.wait_for(lock, - std::chrono::duration(settings.thread_sleep_seconds - + std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); - } - } - - if (shutdown) - break; - } - - std::shared_lock rlock(task->rwlock); - - if (task->removed) - continue; - - try - { - CurrentMetrics::Increment metric_increment{settings.tasks_metric}; - task_result = task->task_function(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - - { - std::unique_lock lock(tasks_mutex); - - if (shutdown) - break; - - --task->concurrent_executors; - - if (task->removed) - continue; - - if (task_result == TaskResult::SUCCESS) - task->count_no_work_done = 0; - else - ++task->count_no_work_done; - - /// If task has done work, it could be executed again immediately. - /// If not, add delay before next run. - - Poco::Timestamp next_time_to_execute; /// current time - if (task_result == TaskResult::ERROR) - next_time_to_execute += 1000000 * (std::min( - settings.task_sleep_seconds_when_no_work_max, - settings.task_sleep_seconds_when_no_work_min * std::pow(settings.task_sleep_seconds_when_no_work_multiplier, task->count_no_work_done)) - + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng)); - else if (task_result == TaskResult::NOTHING_TO_DO) - next_time_to_execute += 1000000 * settings.thread_sleep_seconds_if_nothing_to_do; - - rescheduleTask(task->iterator, next_time_to_execute); - } - } -} - -} diff --git a/src/Storages/MergeTree/BackgroundProcessingPool.h b/src/Storages/MergeTree/BackgroundProcessingPool.h deleted file mode 100644 index dfc83f9e550..00000000000 --- a/src/Storages/MergeTree/BackgroundProcessingPool.h +++ /dev/null @@ -1,163 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace CurrentMetrics -{ - extern const Metric BackgroundPoolTask; -} - -namespace DB -{ - -class BackgroundProcessingPool; -class BackgroundProcessingPoolTaskInfo; - -enum class BackgroundProcessingPoolTaskResult -{ - SUCCESS, - ERROR, - NOTHING_TO_DO, -}; - - -/** Using a fixed number of threads, perform an arbitrary number of tasks in an infinite loop. - * In this case, one task can run simultaneously from different threads. - * Designed for tasks that perform continuous background work (for example, merge). - * `Task` is a function that returns a bool - did it do any work. - * If not, then the next time will be done later. - */ -class BackgroundProcessingPool -{ -public: - /// Returns true, if some useful work was done. In that case, thread will not sleep before next run of this task. - using TaskResult = BackgroundProcessingPoolTaskResult; - using Task = std::function; - using TaskInfo = BackgroundProcessingPoolTaskInfo; - using TaskHandle = std::shared_ptr; - - - struct PoolSettings - { - double thread_sleep_seconds = 10; - double thread_sleep_seconds_random_part = 1.0; - double thread_sleep_seconds_if_nothing_to_do = 0.1; - - /// For exponential backoff. - double task_sleep_seconds_when_no_work_min = 10; - double task_sleep_seconds_when_no_work_max = 600; - double task_sleep_seconds_when_no_work_multiplier = 1.1; - double task_sleep_seconds_when_no_work_random_part = 1.0; - - CurrentMetrics::Metric tasks_metric = CurrentMetrics::BackgroundPoolTask; - - PoolSettings() noexcept {} - }; - - BackgroundProcessingPool(int size_, - const PoolSettings & pool_settings = {}, - const char * log_name = "BackgroundProcessingPool", - const char * thread_name_ = "BackgrProcPool"); - - size_t getNumberOfThreads() const - { - return size; - } - - /// Create task and start it. - TaskHandle addTask(const Task & task); - - /// The following two methods are invoked by Storage*MergeTree at startup - /// Create task but not start it. - TaskHandle createTask(const Task & task); - /// Start the task that was created but not started. Precondition: task was not started. - void startTask(const TaskHandle & task, bool allow_execute_in_parallel = true); - - /// Invoked by Storage*MergeTree at shutdown - void removeTask(const TaskHandle & task); - - ~BackgroundProcessingPool(); - -protected: - friend class BackgroundProcessingPoolTaskInfo; - - using Tasks = std::multimap; /// key is desired next time to execute (priority). - using Threads = std::vector; - - const size_t size; - const char * thread_name; - Poco::Logger * logger; - - Tasks tasks; /// Ordered in priority. - std::mutex tasks_mutex; - - Threads threads; - - bool shutdown{false}; - std::condition_variable wake_event; - - /// Thread group used for profiling purposes - ThreadGroupStatusPtr thread_group; - - void workLoopFunc(); - - void rescheduleTask(Tasks::iterator & task_it, const Poco::Timestamp & new_scheduled_ts) - { - auto node_handle = tasks.extract(task_it); - node_handle.key() = new_scheduled_ts; - task_it = tasks.insert(std::move(node_handle)); - } - -private: - PoolSettings settings; -}; - - -class BackgroundProcessingPoolTaskInfo -{ -public: - /// Signals random idle thread from the pool that this task is ready to be executed. - void signalReadyToRun(); - - BackgroundProcessingPoolTaskInfo(BackgroundProcessingPool & pool_, const BackgroundProcessingPool::Task & function_) - : pool(pool_), task_function(function_) {} - -protected: - friend class BackgroundProcessingPool; - - BackgroundProcessingPool & pool; - BackgroundProcessingPool::Task task_function; - - /// Read lock is held while task is being executed. - /// Write lock is used for stopping BGProcPool - std::shared_mutex rwlock; - - bool allow_execute_in_parallel = false; - size_t concurrent_executors = 0; - - /// Signals that this task must no longer be planned for execution and is about to be removed - std::atomic removed {false}; - - BackgroundProcessingPool::Tasks::iterator iterator; - - /// For exponential backoff. - size_t count_no_work_done = 0; -}; - -} From a535a089d1af493067c7ecb183bcb34968921722 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 19:09:18 +0300 Subject: [PATCH 026/314] Some style fixes --- src/Interpreters/Context.cpp | 32 +++++++++++++++++++ src/Interpreters/Context.h | 3 ++ .../MergeTree/BackgroundJobsExecutor.cpp | 20 ++++++++---- .../MergeTree/BackgroundJobsExecutor.h | 2 -- src/Storages/StorageMergeTree.cpp | 3 +- src/Storages/ya.make | 2 +- 6 files changed, 52 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index e98a8d59155..ac48340c9c6 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -61,6 +61,7 @@ #include #include #include +#include namespace ProfileEvents { @@ -1375,6 +1376,37 @@ BackgroundSchedulePool & Context::getBufferFlushSchedulePool() return *shared->buffer_flush_schedule_pool; } +TaskSleepSettings Context::getBackgroundProcessingTaskSleepSettings() +{ + TaskSleepSettings task_settings; + + const auto & config = getConfigRef(); + task_settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); + task_settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); + task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); + task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); + task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); + task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); + task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); + return task_settings; +} + +TaskSleepSettings Context::getBackgroundMoveTaskSleepSettings() +{ + TaskSleepSettings task_settings; + + const auto & config = getConfigRef(); + task_settings.thread_sleep_seconds = config.getDouble("background_move_processing_pool_thread_sleep_seconds", 10); + task_settings.thread_sleep_seconds_random_part = config.getDouble("background_move_processing_pool_thread_sleep_seconds_random_part", 1.0); + task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_move_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); + task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 10); + task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 600); + task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); + task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); + + return task_settings; +} + BackgroundSchedulePool & Context::getSchedulePool() { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 155d8fbcd73..0f90c26a43c 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -110,6 +110,7 @@ using OutputFormatPtr = std::shared_ptr; class IVolume; using VolumePtr = std::shared_ptr; struct NamedSession; +struct TaskSleepSettings; #if USE_EMBEDDED_COMPILER @@ -506,6 +507,8 @@ public: */ void dropCaches() const; + TaskSleepSettings getBackgroundProcessingTaskSleepSettings(); + TaskSleepSettings getBackgroundMoveTaskSleepSettings(); BackgroundSchedulePool & getBufferFlushSchedulePool(); BackgroundSchedulePool & getSchedulePool(); BackgroundSchedulePool & getMessageBrokerSchedulePool(); diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index c51028862e6..f952b04974b 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -39,27 +39,35 @@ bool incrementIfLess(std::atomic & atomic_value, long max_value) { auto value = atomic_value.load(std::memory_order_relaxed); while (value < max_value) - if(atomic_value.compare_exchange_weak(value, value + 1, std::memory_order_release, std::memory_order_relaxed)) + { + if (atomic_value.compare_exchange_weak(value, value + 1, std::memory_order_release, std::memory_order_relaxed)) return true; + } return false; } } - void IBackgroundJobExecutor::scheduleTask(bool nothing_to_do) { auto errors = errors_count.load(std::memory_order_relaxed); size_t next_time_to_execute = 0; if (errors != 0) + { next_time_to_execute += 1000 * (std::min( sleep_settings.task_sleep_seconds_when_no_work_max, sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, errors)) + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + } else if (nothing_to_do) - next_time_to_execute += 1000 * (sleep_settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + { + next_time_to_execute += 1000 * (sleep_settings.thread_sleep_seconds_if_nothing_to_do + + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + } else + { next_time_to_execute = 1000 * std::uniform_real_distribution(0, sleep_settings.thread_sleep_seconds_random_part)(rng); + } scheduling_task->scheduleAfter(next_time_to_execute); } @@ -138,7 +146,7 @@ void IBackgroundJobExecutor::triggerDataProcessing() scheduling_task->schedule(); } - IBackgroundJobExecutor::~IBackgroundJobExecutor() +IBackgroundJobExecutor::~IBackgroundJobExecutor() { finish(); } @@ -150,7 +158,7 @@ BackgroundJobsExecutor::BackgroundJobsExecutor( data_, global_context_, "(dataProcessingTask)", - TaskSleepSettings{}, + global_context_.getBackgroundProcessingTaskSleepSettings(), {PoolConfig{PoolType::MERGE_MUTATE, global_context_.getSettingsRef().background_pool_size, CurrentMetrics::BackgroundPoolTask}}) { } @@ -170,7 +178,7 @@ BackgroundMovesExecutor::BackgroundMovesExecutor( data_, global_context_, "(dataMovingTask)", - TaskSleepSettings{}, + global_context_.getBackgroundMoveTaskSleepSettings(), {PoolConfig{PoolType::MOVE, global_context_.getSettingsRef().background_move_pool_size, CurrentMetrics::BackgroundMovePoolTask}}) { } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 4d01198f681..bc051702bfb 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -14,7 +14,6 @@ namespace CurrentMetrics namespace DB { - struct TaskSleepSettings { double thread_sleep_seconds = 10; @@ -88,7 +87,6 @@ private: void scheduleTask(bool nothing_to_do); }; - class BackgroundJobsExecutor final : public IBackgroundJobExecutor { public: diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c4f8436678d..b103095eeb2 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -722,7 +722,8 @@ bool StorageMergeTree::merge( return mergeSelectedParts(metadata_snapshot, deduplicate, *merge_mutate_entry); } -bool StorageMergeTree::mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & merge_mutate_entry) { +bool StorageMergeTree::mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & merge_mutate_entry) +{ auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto & future_part = merge_mutate_entry.future_part; Stopwatch stopwatch; diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 107433b5e73..e306787bb92 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -26,7 +26,7 @@ SRCS( LiveView/TemporaryLiveViewCleaner.cpp MergeTree/ActiveDataPartSet.cpp MergeTree/AllMergeSelector.cpp - MergeTree/BackgroundProcessingPool.cpp + MergeTree/BackgroundJobsExecutor.cpp MergeTree/BoolMask.cpp MergeTree/checkDataPart.cpp MergeTree/DataPartsExchange.cpp From 726579251047bda6ae2cbefdf1cd5d8f0492a344 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 21:33:41 +0300 Subject: [PATCH 027/314] Less files --- .../MergeTree/BackgroundJobsExecutor.cpp | 5 --- .../MergeTree/MergeTreeBackgroundJob.h | 37 ------------------- src/Storages/StorageMergeTree.cpp | 1 - 3 files changed, 43 deletions(-) delete mode 100644 src/Storages/MergeTree/MergeTreeBackgroundJob.h diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index f952b04974b..a49712d7d29 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -8,11 +8,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - IBackgroundJobExecutor::IBackgroundJobExecutor( MergeTreeData & data_, Context & global_context_, diff --git a/src/Storages/MergeTree/MergeTreeBackgroundJob.h b/src/Storages/MergeTree/MergeTreeBackgroundJob.h deleted file mode 100644 index 0b1a6ff3a8d..00000000000 --- a/src/Storages/MergeTree/MergeTreeBackgroundJob.h +++ /dev/null @@ -1,37 +0,0 @@ -# pragma once -#include -#include -#include -#include - -namespace DB -{ - -enum PoolType -{ - MERGE_MUTATE, - MOVE, - FETCH, -}; - -struct MergeTreeBackgroundJob -{ - ThreadPool::Job job; - PoolType execute_in_pool; - - MergeTreeBackgroundJob(ThreadPool::Job && job_, PoolType execute_in_pool_) - : job(std::move(job_)), execute_in_pool(execute_in_pool_) - {} - - void operator()() - try - { - job(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -}; - -} diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b103095eeb2..7af3e5b7b40 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -40,7 +40,6 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; extern const int NOT_ENOUGH_SPACE; - extern const int ABORTED; extern const int BAD_ARGUMENTS; extern const int INCORRECT_DATA; extern const int CANNOT_ASSIGN_OPTIMIZE; From ae8ed99805b8ec01bd384e4ffcf702b9c50762e5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 21:35:03 +0300 Subject: [PATCH 028/314] Missed change --- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index a49712d7d29..36a7f8d939e 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -57,7 +57,7 @@ void IBackgroundJobExecutor::scheduleTask(bool nothing_to_do) else if (nothing_to_do) { next_time_to_execute += 1000 * (sleep_settings.thread_sleep_seconds_if_nothing_to_do - + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); } else { From 3e3505258f01e0c0909cc54aa86bec8b9974b572 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 22:33:36 +0300 Subject: [PATCH 029/314] No sleep when we have job --- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 36a7f8d939e..f3b3b6d3110 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -61,7 +61,8 @@ void IBackgroundJobExecutor::scheduleTask(bool nothing_to_do) } else { - next_time_to_execute = 1000 * std::uniform_real_distribution(0, sleep_settings.thread_sleep_seconds_random_part)(rng); + scheduling_task->schedule(); + return; } scheduling_task->scheduleAfter(next_time_to_execute); @@ -108,7 +109,7 @@ try scheduleTask(true); } } -catch (...) /// Exception while we looking for task +catch (...) /// Exception while we looking for a task { tryLogCurrentException(__PRETTY_FUNCTION__); scheduleTask(true); From 3394c22c03517d93629f4607b287e4fb1b437838 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 10:39:27 +0300 Subject: [PATCH 030/314] Fix race and clang-tidy --- src/Interpreters/Context.cpp | 4 ++-- src/Interpreters/Context.h | 4 ++-- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 3 +++ src/Storages/MergeTree/BackgroundJobsExecutor.h | 1 + 4 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index ac48340c9c6..04f9ce3b753 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1376,7 +1376,7 @@ BackgroundSchedulePool & Context::getBufferFlushSchedulePool() return *shared->buffer_flush_schedule_pool; } -TaskSleepSettings Context::getBackgroundProcessingTaskSleepSettings() +TaskSleepSettings Context::getBackgroundProcessingTaskSleepSettings() const { TaskSleepSettings task_settings; @@ -1391,7 +1391,7 @@ TaskSleepSettings Context::getBackgroundProcessingTaskSleepSettings() return task_settings; } -TaskSleepSettings Context::getBackgroundMoveTaskSleepSettings() +TaskSleepSettings Context::getBackgroundMoveTaskSleepSettings() const { TaskSleepSettings task_settings; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 0f90c26a43c..3f70b6557d2 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -507,8 +507,8 @@ public: */ void dropCaches() const; - TaskSleepSettings getBackgroundProcessingTaskSleepSettings(); - TaskSleepSettings getBackgroundMoveTaskSleepSettings(); + TaskSleepSettings getBackgroundProcessingTaskSleepSettings() const; + TaskSleepSettings getBackgroundMoveTaskSleepSettings() const; BackgroundSchedulePool & getBufferFlushSchedulePool(); BackgroundSchedulePool & getSchedulePool(); BackgroundSchedulePool & getMessageBrokerSchedulePool(); diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index f3b3b6d3110..ef594948453 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -117,6 +117,7 @@ catch (...) /// Exception while we looking for a task void IBackgroundJobExecutor::start() { + std::lock_guard lock(task_mutex); if (!scheduling_task) { scheduling_task = global_context.getSchedulePool().createTask( @@ -128,6 +129,7 @@ void IBackgroundJobExecutor::start() void IBackgroundJobExecutor::finish() { + std::lock_guard lock(task_mutex); if (scheduling_task) { scheduling_task->deactivate(); @@ -138,6 +140,7 @@ void IBackgroundJobExecutor::finish() void IBackgroundJobExecutor::triggerDataProcessing() { + std::lock_guard lock(task_mutex); if (scheduling_task) scheduling_task->schedule(); } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index bc051702bfb..c3d2371b9b1 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -65,6 +65,7 @@ private: std::unordered_map pools_configs; BackgroundSchedulePool::TaskHolder scheduling_task; + std::mutex task_mutex; public: IBackgroundJobExecutor( From c5419083e3f998520104116d26912559c2eaf2ab Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 10:43:50 +0300 Subject: [PATCH 031/314] Better method name --- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 2 +- src/Storages/MergeTree/BackgroundJobsExecutor.h | 2 +- src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 6 +++--- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 6 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index ef594948453..4959aeb71f7 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -138,7 +138,7 @@ void IBackgroundJobExecutor::finish() } } -void IBackgroundJobExecutor::triggerDataProcessing() +void IBackgroundJobExecutor::triggerTask() { std::lock_guard lock(task_mutex); if (scheduling_task) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index c3d2371b9b1..66a6031bbf9 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -76,7 +76,7 @@ public: const std::vector & pools_configs_); void start(); - void triggerDataProcessing(); + void triggerTask(); void finish(); virtual ~IBackgroundJobExecutor(); diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index 5ad6a7eebc1..40714e5af31 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -28,7 +28,7 @@ void MergeTreeBlockOutputStream::write(const Block & block) PartLog::addNewPart(storage.global_context, part, watch.elapsed()); /// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'. - storage.background_executor.triggerDataProcessing(); + storage.background_executor.triggerTask(); } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index b7fd7097546..f0732774c3e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -584,7 +584,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper LOG_DEBUG(log, "Pulled {} entries to queue.", copied_entries.size()); } - storage.background_executor.triggerDataProcessing(); + storage.background_executor.triggerTask(); } return stat.version; @@ -668,7 +668,7 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C } if (some_active_mutations_were_killed) - storage.background_executor.triggerDataProcessing(); + storage.background_executor.triggerTask(); if (!entries_to_load.empty()) { @@ -792,7 +792,7 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation( } if (mutation_was_active) - storage.background_executor.triggerDataProcessing(); + storage.background_executor.triggerTask(); return entry; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 7af3e5b7b40..ca22e5d3ed1 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -362,7 +362,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, String current_mutations_by_version.emplace(version, insertion.first->second); LOG_INFO(log, "Added mutation: {}", mutation_file_name); - background_executor.triggerDataProcessing(); + background_executor.triggerTask(); return version; } @@ -580,7 +580,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) } /// Maybe there is another mutation that was blocked by the killed one. Try to execute it immediately. - background_executor.triggerDataProcessing(); + background_executor.triggerTask(); return CancellationCode::CancelSent; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index bc55b87fee7..bf10d931f39 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5868,7 +5868,7 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI { auto lock = queue.lockQueue(); - background_executor.triggerDataProcessing(); + background_executor.triggerTask(); /// This is significant, because the execution of this task could be delayed at BackgroundPool. /// And we force it to be executed. } From 583d7042e99d822274e0e1fb598b72fb3fda1696 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 11:29:18 +0300 Subject: [PATCH 032/314] Get rid of some deadlocks --- .../MergeTree/BackgroundJobsExecutor.cpp | 3 --- .../MergeTree/BackgroundJobsExecutor.h | 1 - src/Storages/StorageReplicatedMergeTree.cpp | 19 +++++-------------- 3 files changed, 5 insertions(+), 18 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 4959aeb71f7..c3b351af520 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -117,7 +117,6 @@ catch (...) /// Exception while we looking for a task void IBackgroundJobExecutor::start() { - std::lock_guard lock(task_mutex); if (!scheduling_task) { scheduling_task = global_context.getSchedulePool().createTask( @@ -129,7 +128,6 @@ void IBackgroundJobExecutor::start() void IBackgroundJobExecutor::finish() { - std::lock_guard lock(task_mutex); if (scheduling_task) { scheduling_task->deactivate(); @@ -140,7 +138,6 @@ void IBackgroundJobExecutor::finish() void IBackgroundJobExecutor::triggerTask() { - std::lock_guard lock(task_mutex); if (scheduling_task) scheduling_task->schedule(); } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 66a6031bbf9..0abb5bce011 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -65,7 +65,6 @@ private: std::unordered_map pools_configs; BackgroundSchedulePool::TaskHolder scheduling_task; - std::mutex task_mutex; public: IBackgroundJobExecutor( diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index bf10d931f39..4cf690c78d1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3482,10 +3482,7 @@ void StorageReplicatedMergeTree::startup() /// If we don't separate create/start steps, race condition will happen /// between the assignment of queue_task_handle and queueTask that use the queue_task_handle. - { - auto lock = queue.lockQueue(); - background_executor.start(); - } + background_executor.start(); if (areBackgroundMovesNeeded()) background_moves_executor.start(); @@ -3518,13 +3515,10 @@ void StorageReplicatedMergeTree::shutdown() parts_mover.moves_blocker.cancelForever(); restarting_thread.shutdown(); + background_executor.finish(); { - /// Queue can trigger queue_task_handle itself. So we ensure that all - /// queue processes finished and after that reset queue_task_handle. auto lock = queue.lockQueue(); - background_executor.finish(); - /// Cancel logs pulling after background task were cancelled. It's still /// required because we can trigger pullLogsToQueue during manual OPTIMIZE, /// MUTATE, etc. query. @@ -5866,12 +5860,9 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI /// Let's fetch new log entries firstly queue.pullLogsToQueue(getZooKeeper()); - { - auto lock = queue.lockQueue(); - background_executor.triggerTask(); - /// This is significant, because the execution of this task could be delayed at BackgroundPool. - /// And we force it to be executed. - } + /// This is significant, because the execution of this task could be delayed at BackgroundPool. + /// And we force it to be executed. + background_executor.triggerTask(); Poco::Event target_size_event; auto callback = [&target_size_event, queue_size] (size_t new_queue_size) From e84eda176a91bcf5c9b5de4d8308d947f248588c Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 13:22:02 +0300 Subject: [PATCH 033/314] Add test and slightly refactored --- .../MergeTree/BackgroundJobsExecutor.cpp | 28 +++++---- .../MergeTree/BackgroundJobsExecutor.h | 22 ++++--- .../tests/gtest_background_executor.cpp | 62 +++++++++++++++++++ 3 files changed, 91 insertions(+), 21 deletions(-) create mode 100644 src/Storages/tests/gtest_background_executor.cpp diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index c3b351af520..12c3ff3e418 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -9,14 +9,10 @@ namespace DB { IBackgroundJobExecutor::IBackgroundJobExecutor( - MergeTreeData & data_, Context & global_context_, - const String & task_name_, const TaskSleepSettings & sleep_settings_, const std::vector & pools_configs_) - : data(data_) - , global_context(global_context_) - , task_name(task_name_) + : global_context(global_context_) , sleep_settings(sleep_settings_) , rng(randomSeed()) { @@ -30,7 +26,7 @@ IBackgroundJobExecutor::IBackgroundJobExecutor( namespace { -bool incrementIfLess(std::atomic & atomic_value, long max_value) +bool incrementIfLess(std::atomic & atomic_value, Int64 max_value) { auto value = atomic_value.load(std::memory_order_relaxed); while (value < max_value) @@ -120,7 +116,7 @@ void IBackgroundJobExecutor::start() if (!scheduling_task) { scheduling_task = global_context.getSchedulePool().createTask( - data.getStorageID().getFullTableName() + task_name, [this]{ jobExecutingTask(); }); + getBackgroundJobName(), [this]{ jobExecutingTask(); }); } scheduling_task->activateAndSchedule(); @@ -151,14 +147,18 @@ BackgroundJobsExecutor::BackgroundJobsExecutor( MergeTreeData & data_, Context & global_context_) : IBackgroundJobExecutor( - data_, global_context_, - "(dataProcessingTask)", global_context_.getBackgroundProcessingTaskSleepSettings(), {PoolConfig{PoolType::MERGE_MUTATE, global_context_.getSettingsRef().background_pool_size, CurrentMetrics::BackgroundPoolTask}}) + , data(data_) { } +String BackgroundJobsExecutor::getBackgroundJobName() const +{ + return data.getStorageID().getFullTableName() + " (dataProcessingTask)"; +} + std::optional BackgroundJobsExecutor::getBackgroundJob() { auto job = data.getDataProcessingJob(); @@ -171,14 +171,18 @@ BackgroundMovesExecutor::BackgroundMovesExecutor( MergeTreeData & data_, Context & global_context_) : IBackgroundJobExecutor( - data_, global_context_, - "(dataMovingTask)", global_context_.getBackgroundMoveTaskSleepSettings(), - {PoolConfig{PoolType::MOVE, global_context_.getSettingsRef().background_move_pool_size, CurrentMetrics::BackgroundMovePoolTask}}) + {PoolConfig{PoolType::MOVE, global_context_.getSettingsRef().background_move_pool_size, CurrentMetrics::BackgroundMovePoolTask}}) + , data(data_) { } +String BackgroundMovesExecutor::getBackgroundJobName() const +{ + return data.getStorageID().getFullTableName() + " (dataMovingTask)"; +} + std::optional BackgroundMovesExecutor::getBackgroundJob() { auto job = data.getDataMovingJob(); diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 0abb5bce011..1710cccc54c 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -51,8 +51,6 @@ struct JobAndPool class IBackgroundJobExecutor { -protected: - MergeTreeData & data; Context & global_context; private: String task_name; @@ -67,13 +65,6 @@ private: BackgroundSchedulePool::TaskHolder scheduling_task; public: - IBackgroundJobExecutor( - MergeTreeData & data_, - Context & global_context_, - const String & task_name_, - const TaskSleepSettings & sleep_settings_, - const std::vector & pools_configs_); - void start(); void triggerTask(); void finish(); @@ -81,7 +72,14 @@ public: virtual ~IBackgroundJobExecutor(); protected: + IBackgroundJobExecutor( + Context & global_context_, + const TaskSleepSettings & sleep_settings_, + const std::vector & pools_configs_); + + virtual String getBackgroundJobName() const = 0; virtual std::optional getBackgroundJob() = 0; + private: void jobExecutingTask(); void scheduleTask(bool nothing_to_do); @@ -89,23 +87,29 @@ private: class BackgroundJobsExecutor final : public IBackgroundJobExecutor { +private: + MergeTreeData & data; public: BackgroundJobsExecutor( MergeTreeData & data_, Context & global_context_); protected: + String getBackgroundJobName() const override; std::optional getBackgroundJob() override; }; class BackgroundMovesExecutor final : public IBackgroundJobExecutor { +private: + MergeTreeData & data; public: BackgroundMovesExecutor( MergeTreeData & data_, Context & global_context_); protected: + String getBackgroundJobName() const override; std::optional getBackgroundJob() override; }; diff --git a/src/Storages/tests/gtest_background_executor.cpp b/src/Storages/tests/gtest_background_executor.cpp new file mode 100644 index 00000000000..1bc96ed0c42 --- /dev/null +++ b/src/Storages/tests/gtest_background_executor.cpp @@ -0,0 +1,62 @@ +#include +#include +#include +#include +#include +#include +using namespace std::chrono_literals; +namespace CurrentMetrics +{ + extern const Metric BackgroundPoolTask; +} + +using namespace DB; + +static std::atomic counter{0}; + +class TestJobExecutor : public IBackgroundJobExecutor +{ +public: + TestJobExecutor(Context & context) + :IBackgroundJobExecutor( + context, + TaskSleepSettings{}, + {PoolConfig{PoolType::MERGE_MUTATE, 4, CurrentMetrics::BackgroundPoolTask}}) + {} + +protected: + String getBackgroundJobName() const override + { + return "TestJob"; + } + + std::optional getBackgroundJob() override + { + return JobAndPool{[] { std::this_thread::sleep_for(1s); counter++; }, PoolType::MERGE_MUTATE}; + } +}; + +using TestExecutorPtr = std::unique_ptr; + +TEST(BackgroundExecutor, TestMetric) +{ + auto & context_holder = getContext(); + std::vector executors; + for (size_t i = 0; i < 100; ++i) + executors.emplace_back(std::make_unique(const_cast(context_holder.context))); + + for (size_t i = 0; i < 100; ++i) + executors[i]->start(); + + for (size_t i = 0; i < 100; ++i) + { + EXPECT_TRUE(CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load() <= 4); + std::this_thread::sleep_for(200ms); + } + + for (size_t i = 0; i < 100; ++i) + executors[i]->finish(); + + /// Sanity check + EXPECT_TRUE(counter > 50); +} From 374949051c9da45b1835440045d6313d61f11d10 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 13:41:36 +0300 Subject: [PATCH 034/314] Add thread safety --- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 3 +++ src/Storages/MergeTree/BackgroundJobsExecutor.h | 1 + 2 files changed, 4 insertions(+) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 12c3ff3e418..873d7fd8aa0 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -113,6 +113,7 @@ catch (...) /// Exception while we looking for a task void IBackgroundJobExecutor::start() { + std::lock_guard lock(scheduling_task_mutex); if (!scheduling_task) { scheduling_task = global_context.getSchedulePool().createTask( @@ -124,6 +125,7 @@ void IBackgroundJobExecutor::start() void IBackgroundJobExecutor::finish() { + std::lock_guard lock(scheduling_task_mutex); if (scheduling_task) { scheduling_task->deactivate(); @@ -134,6 +136,7 @@ void IBackgroundJobExecutor::finish() void IBackgroundJobExecutor::triggerTask() { + std::lock_guard lock(scheduling_task_mutex); if (scheduling_task) scheduling_task->schedule(); } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 1710cccc54c..03d3823bcab 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -63,6 +63,7 @@ private: std::unordered_map pools_configs; BackgroundSchedulePool::TaskHolder scheduling_task; + std::mutex scheduling_task_mutex; public: void start(); From 84b0c2204491874190d43696d47cd18a731dbe22 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 13:54:50 +0300 Subject: [PATCH 035/314] fix possible deadlock --- src/Storages/StorageMergeTree.cpp | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ca22e5d3ed1..b9a675ffedc 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -352,16 +352,18 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, String /// where storage can be placed. See loadMutations(). auto disk = getStoragePolicy()->getAnyDisk(); Int64 version; - std::lock_guard lock(currently_processing_in_background_mutex); + { + std::lock_guard lock(currently_processing_in_background_mutex); - MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get()); - version = increment.get(); - entry.commit(version); - mutation_file_name = entry.file_name; - auto insertion = current_mutations_by_id.emplace(mutation_file_name, std::move(entry)); - current_mutations_by_version.emplace(version, insertion.first->second); + MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get()); + version = increment.get(); + entry.commit(version); + mutation_file_name = entry.file_name; + auto insertion = current_mutations_by_id.emplace(mutation_file_name, std::move(entry)); + current_mutations_by_version.emplace(version, insertion.first->second); - LOG_INFO(log, "Added mutation: {}", mutation_file_name); + LOG_INFO(log, "Added mutation: {}", mutation_file_name); + } background_executor.triggerTask(); return version; } From dccf51b3169480907b15b7049b8c38152fe94c21 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 14:08:39 +0300 Subject: [PATCH 036/314] Remove redundant logging --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 6aa3d1f3238..90631762548 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -165,11 +165,8 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge() const UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used) const { - //LOG_DEBUG(&Poco::Logger::get("DEBUG"), "POOL SIZE {}, POOL USED {}", pool_size, pool_used); if (pool_used > pool_size) - { throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR); - } size_t free_entries = pool_size - pool_used; const auto data_settings = data.getSettings(); From 629506445dfd1f155a4c09b9c460270ef7081119 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 16:57:50 +0300 Subject: [PATCH 037/314] Better backoff policy --- .../MergeTree/BackgroundJobsExecutor.cpp | 32 ++- .../MergeTree/BackgroundJobsExecutor.h | 4 +- .../MergeTree/BackgroundProcessingPool.cpp | 243 ------------------ .../tests/gtest_background_executor.cpp | 4 +- 4 files changed, 19 insertions(+), 264 deletions(-) delete mode 100644 src/Storages/MergeTree/BackgroundProcessingPool.cpp diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 873d7fd8aa0..57003bbfaa0 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -39,21 +39,17 @@ bool incrementIfLess(std::atomic & atomic_value, Int64 max_value) } -void IBackgroundJobExecutor::scheduleTask(bool nothing_to_do) +void IBackgroundJobExecutor::scheduleTask() { - auto errors = errors_count.load(std::memory_order_relaxed); - size_t next_time_to_execute = 0; - if (errors != 0) + auto no_work_done_times = no_work_done_count.load(std::memory_order_relaxed); + if (no_work_done_times != 0) { - next_time_to_execute += 1000 * (std::min( + auto next_time_to_execute = 1000 * (std::min( sleep_settings.task_sleep_seconds_when_no_work_max, - sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, errors)) - + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); - } - else if (nothing_to_do) - { - next_time_to_execute += 1000 * (sleep_settings.thread_sleep_seconds_if_nothing_to_do + sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + + scheduling_task->scheduleAfter(next_time_to_execute); } else { @@ -61,7 +57,6 @@ void IBackgroundJobExecutor::scheduleTask(bool nothing_to_do) return; } - scheduling_task->scheduleAfter(next_time_to_execute); } void IBackgroundJobExecutor::jobExecutingTask() @@ -82,11 +77,11 @@ try { job(); CurrentMetrics::values[pool_config.tasks_metric]--; - errors_count = 0; + no_work_done_count = 0; } catch (...) { - errors_count++; + no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; } @@ -94,21 +89,24 @@ try } catch (...) { + no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; } } - scheduleTask(false); } else /// Nothing to do, no jobs { - scheduleTask(true); + no_work_done_count++; } + + scheduleTask(); } catch (...) /// Exception while we looking for a task { + no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); - scheduleTask(true); + scheduleTask(); } void IBackgroundJobExecutor::start() diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 03d3823bcab..d12124492f9 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -57,7 +57,7 @@ private: TaskSleepSettings sleep_settings; pcg64 rng; - std::atomic errors_count{0}; + std::atomic no_work_done_count{0}; std::unordered_map pools; std::unordered_map pools_configs; @@ -83,7 +83,7 @@ protected: private: void jobExecutingTask(); - void scheduleTask(bool nothing_to_do); + void scheduleTask(); }; class BackgroundJobsExecutor final : public IBackgroundJobExecutor diff --git a/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/src/Storages/MergeTree/BackgroundProcessingPool.cpp deleted file mode 100644 index d779bd16814..00000000000 --- a/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ /dev/null @@ -1,243 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - - -namespace DB -{ - -void BackgroundProcessingPoolTaskInfo::signalReadyToRun() -{ - Poco::Timestamp current_time; - { - std::unique_lock lock(pool.tasks_mutex); - - /// This check ensures that the iterator is valid. Must be performed under the same mutex as invalidation. - if (removed) - return; - - /// If this task did nothing the previous time and still should sleep, then reschedule to cancel the sleep. - const auto & scheduled_time = iterator->first; - if (scheduled_time > current_time) - pool.rescheduleTask(iterator, current_time); - - /// Note that if all threads are currently busy doing their work, this call will not wakeup any thread. - pool.wake_event.notify_one(); - } -} - - -BackgroundProcessingPool::BackgroundProcessingPool(int size_, - const PoolSettings & pool_settings, - const char * log_name, - const char * thread_name_) - : size(size_) - , thread_name(thread_name_) - , settings(pool_settings) -{ - logger = &Poco::Logger::get(log_name); - LOG_INFO(logger, "Create {} with {} threads", log_name, size); - - threads.resize(size); - for (auto & thread : threads) - thread = ThreadFromGlobalPool([this] { workLoopFunc(); }); -} - - -BackgroundProcessingPool::TaskHandle BackgroundProcessingPool::createTask(const Task & task) -{ - return std::make_shared(*this, task); -} - -void BackgroundProcessingPool::startTask(const TaskHandle & task, bool allow_execute_in_parallel) -{ - Poco::Timestamp current_time; - - task->allow_execute_in_parallel = allow_execute_in_parallel; - - { - std::unique_lock lock(tasks_mutex); - task->iterator = tasks.emplace(current_time, task); - - wake_event.notify_all(); - } - -} - -BackgroundProcessingPool::TaskHandle BackgroundProcessingPool::addTask(const Task & task) -{ - TaskHandle res = createTask(task); - startTask(res); - return res; -} - -void BackgroundProcessingPool::removeTask(const TaskHandle & task) -{ - if (task->removed.exchange(true)) - return; - - /// Wait for all executions of this task. - { - std::unique_lock wlock(task->rwlock); - } - - { - std::unique_lock lock(tasks_mutex); - tasks.erase(task->iterator); - /// Note that the task may be still accessible through TaskHandle (shared_ptr). - } -} - -BackgroundProcessingPool::~BackgroundProcessingPool() -{ - try - { - { - std::lock_guard lock(tasks_mutex); - shutdown = true; - wake_event.notify_all(); - } - - for (auto & thread : threads) - thread.join(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} - - -void BackgroundProcessingPool::workLoopFunc() -{ - setThreadName(thread_name); - - { - std::lock_guard lock(tasks_mutex); - - if (thread_group) - { - /// Put all threads to one thread pool - CurrentThread::attachTo(thread_group); - } - else - { - CurrentThread::initializeQuery(); - thread_group = CurrentThread::getGroup(); - } - } - - SCOPE_EXIT({ CurrentThread::detachQueryIfNotDetached(); }); - - pcg64 rng(randomSeed()); - std::this_thread::sleep_for(std::chrono::duration(std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); - - Poco::Timestamp scheduled_task_start_time; - - while (true) - { - TaskResult task_result = TaskResult::ERROR; - TaskHandle task; - - { - std::unique_lock lock(tasks_mutex); - - while (!task && !shutdown) - { - for (const auto & [time, handle] : tasks) - { - if (!handle->removed - && (handle->allow_execute_in_parallel || handle->concurrent_executors == 0)) - { - task = handle; - scheduled_task_start_time = time; - ++task->concurrent_executors; - break; - } - } - - if (task) - { - Poco::Timestamp current_time; - - if (scheduled_task_start_time <= current_time) - continue; - - wake_event.wait_for(lock, - std::chrono::microseconds(scheduled_task_start_time - current_time - + std::uniform_int_distribution(0, settings.thread_sleep_seconds_random_part * 1000000)(rng))); - } - else - { - wake_event.wait_for(lock, - std::chrono::duration(settings.thread_sleep_seconds - + std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); - } - } - - if (shutdown) - break; - } - - std::shared_lock rlock(task->rwlock); - - if (task->removed) - continue; - - try - { - CurrentMetrics::Increment metric_increment{settings.tasks_metric}; - task_result = task->task_function(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - - { - std::unique_lock lock(tasks_mutex); - - if (shutdown) - break; - - --task->concurrent_executors; - - if (task->removed) - continue; - - if (task_result == TaskResult::SUCCESS) - task->count_no_work_done = 0; - else if (task_result == TaskResult::ERROR) - ++task->count_no_work_done; - /// NOTHING_TO_DO should not increment count_no_work_done - /// otherwise error after period of inactivity (lot of NOTHING_TO_DO) - /// leads to 5-10 min replication hang - - /// If task has done work, it could be executed again immediately. - /// If not, add delay before next run. - - Poco::Timestamp next_time_to_execute; /// current time - if (task_result == TaskResult::ERROR) - next_time_to_execute += 1000000 * (std::min( - settings.task_sleep_seconds_when_no_work_max, - settings.task_sleep_seconds_when_no_work_min * std::pow(settings.task_sleep_seconds_when_no_work_multiplier, task->count_no_work_done)) - + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng)); - else if (task_result == TaskResult::NOTHING_TO_DO) - next_time_to_execute += 1000000 * settings.thread_sleep_seconds_if_nothing_to_do; - - rescheduleTask(task->iterator, next_time_to_execute); - } - } -} - -} diff --git a/src/Storages/tests/gtest_background_executor.cpp b/src/Storages/tests/gtest_background_executor.cpp index 1bc96ed0c42..77bc539b1b1 100644 --- a/src/Storages/tests/gtest_background_executor.cpp +++ b/src/Storages/tests/gtest_background_executor.cpp @@ -17,7 +17,7 @@ static std::atomic counter{0}; class TestJobExecutor : public IBackgroundJobExecutor { public: - TestJobExecutor(Context & context) + explicit TestJobExecutor(Context & context) :IBackgroundJobExecutor( context, TaskSleepSettings{}, @@ -40,7 +40,7 @@ using TestExecutorPtr = std::unique_ptr; TEST(BackgroundExecutor, TestMetric) { - auto & context_holder = getContext(); + const auto & context_holder = getContext(); std::vector executors; for (size_t i = 0; i < 100; ++i) executors.emplace_back(std::make_unique(const_cast(context_holder.context))); From 4c1f5cab137e05533d1b8505b0c40a19a86dd8e6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 19:10:22 +0300 Subject: [PATCH 038/314] Sleep less in background task --- src/Interpreters/Context.cpp | 12 ++++++------ src/Interpreters/InterpreterSystemQuery.cpp | 11 +++++++++-- src/Storages/IStorage.h | 2 ++ src/Storages/MergeTree/BackgroundJobsExecutor.h | 6 +++--- src/Storages/StorageMergeTree.cpp | 8 ++++++++ src/Storages/StorageMergeTree.h | 2 ++ src/Storages/StorageReplicatedMergeTree.cpp | 9 +++++++++ src/Storages/StorageReplicatedMergeTree.h | 2 ++ 8 files changed, 41 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 47068002b85..5b383968435 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1384,9 +1384,9 @@ TaskSleepSettings Context::getBackgroundProcessingTaskSleepSettings() const task_settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); task_settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); - task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); - task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); + task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 5); + task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 300); + task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.01); task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); return task_settings; } @@ -1399,9 +1399,9 @@ TaskSleepSettings Context::getBackgroundMoveTaskSleepSettings() const task_settings.thread_sleep_seconds = config.getDouble("background_move_processing_pool_thread_sleep_seconds", 10); task_settings.thread_sleep_seconds_random_part = config.getDouble("background_move_processing_pool_thread_sleep_seconds_random_part", 1.0); task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_move_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 10); - task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 600); - task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); + task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 5); + task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 300); + task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.01); task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); return task_settings; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 4bfa84090c2..4d03e1871ef 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -135,11 +135,15 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type, if (table_id) { + auto table = DatabaseCatalog::instance().tryGetTable(table_id, context); context.checkAccess(getRequiredAccessType(action_type), table_id); if (start) - manager->remove(table_id, action_type); + { + manager->remove(table, action_type); + table->onActionLockRemove(action_type); + } else - manager->add(table_id, action_type); + manager->add(table, action_type); } else { @@ -164,7 +168,10 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type, } if (start) + { manager->remove(table, action_type); + table->onActionLockRemove(action_type); + } else manager->add(table, action_type); } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 4a2e70aa84b..cced739863a 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -428,6 +428,8 @@ public: return {}; } + virtual void onActionLockRemove(StorageActionBlockType /* action_type */) {} + std::atomic is_dropped{false}; /// Does table support index for IN sections diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index d12124492f9..aa5eeaf2ded 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -21,9 +21,9 @@ struct TaskSleepSettings double thread_sleep_seconds_if_nothing_to_do = 0.1; /// For exponential backoff. - double task_sleep_seconds_when_no_work_min = 10; - double task_sleep_seconds_when_no_work_max = 600; - double task_sleep_seconds_when_no_work_multiplier = 1.1; + double task_sleep_seconds_when_no_work_min = 5; + double task_sleep_seconds_when_no_work_max = 300; + double task_sleep_seconds_when_no_work_multiplier = 1.01; double task_sleep_seconds_when_no_work_random_part = 1.0; }; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b9a675ffedc..19c8357f2b5 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1388,6 +1388,14 @@ ActionLock StorageMergeTree::getActionLock(StorageActionBlockType action_type) return {}; } +void StorageMergeTree::onActionLockRemove(StorageActionBlockType action_type) +{ + if (action_type == ActionLocks::PartsMerge || action_type == ActionLocks::PartsTTLMerge) + background_executor.triggerTask(); + else if (action_type == ActionLocks::PartsMove) + background_moves_executor.triggerTask(); +} + CheckResults StorageMergeTree::checkData(const ASTPtr & query, const Context & context) { CheckResults results; diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index f7473872189..e6892075da8 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -85,6 +85,8 @@ public: ActionLock getActionLock(StorageActionBlockType action_type) override; + void onActionLockRemove(StorageActionBlockType action_type) override; + CheckResults checkData(const ASTPtr & query, const Context & context) override; ThreadPool::Job getDataProcessingJob() override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 4cf690c78d1..dedaa3c0cf0 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5852,6 +5852,15 @@ ActionLock StorageReplicatedMergeTree::getActionLock(StorageActionBlockType acti return {}; } +void StorageReplicatedMergeTree::onActionLockRemove(StorageActionBlockType action_type) +{ + if (action_type == ActionLocks::PartsMerge || action_type == ActionLocks::PartsTTLMerge + || action_type == ActionLocks::PartsFetch || action_type == ActionLocks::PartsSend + || action_type == ActionLocks::ReplicationQueue) + background_executor.triggerTask(); + else if (action_type == ActionLocks::PartsMove) + background_moves_executor.triggerTask(); +} bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UInt64 max_wait_milliseconds) { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index ba01ca9d3af..e9c54524892 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -138,6 +138,8 @@ public: ActionLock getActionLock(StorageActionBlockType action_type) override; + void onActionLockRemove(StorageActionBlockType action_type) override; + /// Wait when replication queue size becomes less or equal than queue_size /// If timeout is exceeded returns false bool waitForShrinkingQueueSize(size_t queue_size = 0, UInt64 max_wait_milliseconds = 0); From 2b2a55a0818406739e2cd923553f4f6ddecb51f7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 22:23:46 +0300 Subject: [PATCH 039/314] Add action lock only if table exists --- src/Interpreters/InterpreterSystemQuery.cpp | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 4d03e1871ef..2774da7201a 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -136,14 +136,16 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type, if (table_id) { auto table = DatabaseCatalog::instance().tryGetTable(table_id, context); - context.checkAccess(getRequiredAccessType(action_type), table_id); - if (start) + if (table) { - manager->remove(table, action_type); - table->onActionLockRemove(action_type); + if (start) + { + manager->remove(table, action_type); + table->onActionLockRemove(action_type); + } + else + manager->add(table, action_type); } - else - manager->add(table, action_type); } else { From 0fb1f74b2ed340c2ea68e89fde5f5620d39b8c3f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 16 Oct 2020 11:00:15 +0300 Subject: [PATCH 040/314] Trying to schedule better --- src/Interpreters/Context.cpp | 12 ++++++------ src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 5 +++-- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5b383968435..47068002b85 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1384,9 +1384,9 @@ TaskSleepSettings Context::getBackgroundProcessingTaskSleepSettings() const task_settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); task_settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 5); - task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 300); - task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.01); + task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); + task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); + task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); return task_settings; } @@ -1399,9 +1399,9 @@ TaskSleepSettings Context::getBackgroundMoveTaskSleepSettings() const task_settings.thread_sleep_seconds = config.getDouble("background_move_processing_pool_thread_sleep_seconds", 10); task_settings.thread_sleep_seconds_random_part = config.getDouble("background_move_processing_pool_thread_sleep_seconds_random_part", 1.0); task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_move_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 5); - task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 300); - task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.01); + task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 10); + task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 600); + task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); return task_settings; diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 57003bbfaa0..5825fbec939 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -54,7 +54,6 @@ void IBackgroundJobExecutor::scheduleTask() else { scheduling_task->schedule(); - return; } } @@ -85,6 +84,7 @@ try tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; } + scheduleTask(); }); } catch (...) @@ -92,15 +92,16 @@ try no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; + scheduleTask(); } } } else /// Nothing to do, no jobs { no_work_done_count++; + scheduleTask(); } - scheduleTask(); } catch (...) /// Exception while we looking for a task { From 7edbe4d276567db6a0ad653a4020723899c06cd5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 16 Oct 2020 13:12:31 +0300 Subject: [PATCH 041/314] Add some comments --- src/Interpreters/Context.cpp | 8 +- src/Interpreters/Context.h | 8 +- src/Storages/IStorage.h | 1 + .../MergeTree/BackgroundJobsExecutor.cpp | 89 +++++++++++-------- .../MergeTree/BackgroundJobsExecutor.h | 81 ++++++++++++----- src/Storages/MergeTree/MergeTreeData.cpp | 6 +- src/Storages/MergeTree/MergeTreeData.h | 7 +- src/Storages/StorageMergeTree.cpp | 10 +-- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 7 +- src/Storages/StorageReplicatedMergeTree.h | 2 +- 11 files changed, 136 insertions(+), 85 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 47068002b85..e47560319ab 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1376,9 +1376,9 @@ BackgroundSchedulePool & Context::getBufferFlushSchedulePool() return *shared->buffer_flush_schedule_pool; } -TaskSleepSettings Context::getBackgroundProcessingTaskSleepSettings() const +BackgroundTaskSchedulingSettings Context::getBackgroundProcessingTaskSchedulingSettings() const { - TaskSleepSettings task_settings; + BackgroundTaskSchedulingSettings task_settings; const auto & config = getConfigRef(); task_settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); @@ -1391,9 +1391,9 @@ TaskSleepSettings Context::getBackgroundProcessingTaskSleepSettings() const return task_settings; } -TaskSleepSettings Context::getBackgroundMoveTaskSleepSettings() const +BackgroundTaskSchedulingSettings Context::getBackgroundMoveTaskSchedulingSettings() const { - TaskSleepSettings task_settings; + BackgroundTaskSchedulingSettings task_settings; const auto & config = getConfigRef(); task_settings.thread_sleep_seconds = config.getDouble("background_move_processing_pool_thread_sleep_seconds", 10); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 3f70b6557d2..66569d48c08 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -110,7 +110,7 @@ using OutputFormatPtr = std::shared_ptr; class IVolume; using VolumePtr = std::shared_ptr; struct NamedSession; -struct TaskSleepSettings; +struct BackgroundTaskSchedulingSettings; #if USE_EMBEDDED_COMPILER @@ -507,8 +507,10 @@ public: */ void dropCaches() const; - TaskSleepSettings getBackgroundProcessingTaskSleepSettings() const; - TaskSleepSettings getBackgroundMoveTaskSleepSettings() const; + /// Settings for MergeTree background tasks stored in config.xml + BackgroundTaskSchedulingSettings getBackgroundProcessingTaskSchedulingSettings() const; + BackgroundTaskSchedulingSettings getBackgroundMoveTaskSchedulingSettings() const; + BackgroundSchedulePool & getBufferFlushSchedulePool(); BackgroundSchedulePool & getSchedulePool(); BackgroundSchedulePool & getMessageBrokerSchedulePool(); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index cced739863a..6a5d9b3ba9a 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -428,6 +428,7 @@ public: return {}; } + /// Call when lock from previous method removed virtual void onActionLockRemove(StorageActionBlockType /* action_type */) {} std::atomic is_dropped{false}; diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 5825fbec939..b888eec3dfe 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -5,12 +5,18 @@ #include #include +namespace CurrentMetrics +{ + extern const Metric BackgroundPoolTask; + extern const Metric BackgroundMovePoolTask; +} + namespace DB { IBackgroundJobExecutor::IBackgroundJobExecutor( Context & global_context_, - const TaskSleepSettings & sleep_settings_, + const BackgroundTaskSchedulingSettings & sleep_settings_, const std::vector & pools_configs_) : global_context(global_context_) , sleep_settings(sleep_settings_) @@ -23,10 +29,35 @@ IBackgroundJobExecutor::IBackgroundJobExecutor( } } + +void IBackgroundJobExecutor::scheduleTask() +{ + auto no_work_done_times = no_work_done_count.load(std::memory_order_relaxed); + /// If we have no jobs or some errors than sleep with backoff + if (no_work_done_times != 0) + { + auto next_time_to_execute = 1000 * (std::min( + sleep_settings.task_sleep_seconds_when_no_work_max, + sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) + + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + + scheduling_task->scheduleAfter(next_time_to_execute); + } + else + { + /// We have background jobs, schedule task as soon as possible + scheduling_task->schedule(); + } + +} + namespace { -bool incrementIfLess(std::atomic & atomic_value, Int64 max_value) +/// Tricky function: we have separate thread pool with max_threads in each background executor for each table +/// But we want total background threads to be less than max_threads value. So we use global atomic counter (BackgroundMetric) +/// to limit total number of background threads. +bool incrementMetricIfLessThanMax(std::atomic & atomic_value, Int64 max_value) { auto value = atomic_value.load(std::memory_order_relaxed); while (value < max_value) @@ -39,34 +70,15 @@ bool incrementIfLess(std::atomic & atomic_value, Int64 max_value) } -void IBackgroundJobExecutor::scheduleTask() -{ - auto no_work_done_times = no_work_done_count.load(std::memory_order_relaxed); - if (no_work_done_times != 0) - { - auto next_time_to_execute = 1000 * (std::min( - sleep_settings.task_sleep_seconds_when_no_work_max, - sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) - + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); - - scheduling_task->scheduleAfter(next_time_to_execute); - } - else - { - scheduling_task->schedule(); - } - -} - void IBackgroundJobExecutor::jobExecutingTask() try { auto job_and_pool = getBackgroundJob(); - if (job_and_pool) + if (job_and_pool) /// If we have job, than try to assign into background pool { auto & pool_config = pools_configs[job_and_pool->pool_type]; - /// If corresponding pool is not full, otherwise try next time - if (incrementIfLess(CurrentMetrics::values[pool_config.tasks_metric], pool_config.max_pool_size)) + /// If corresponding pool is not full increment metric and assign new job + if (incrementMetricIfLessThanMax(CurrentMetrics::values[pool_config.tasks_metric], pool_config.max_pool_size)) { try /// this try required because we have to manually decrement metric { @@ -75,6 +87,7 @@ try try /// We don't want exceptions in background pool { job(); + /// Job done, decrement metric and reset no_work counter CurrentMetrics::values[pool_config.tasks_metric]--; no_work_done_count = 0; } @@ -84,17 +97,23 @@ try tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; } + /// Job done, new empty space in pool, schedule background task scheduleTask(); }); } catch (...) { + /// With our Pool settings scheduleOrThrowOnError shouldn't throw exceptions, but for safety add catch here no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; scheduleTask(); } } + else /// Pool is full and we have some work to do, let's try to schedule without backoff + { + scheduleTask(); + } } else /// Nothing to do, no jobs { @@ -103,7 +122,7 @@ try } } -catch (...) /// Exception while we looking for a task +catch (...) /// Exception while we looking for a task, reschedule { no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); @@ -116,7 +135,7 @@ void IBackgroundJobExecutor::start() if (!scheduling_task) { scheduling_task = global_context.getSchedulePool().createTask( - getBackgroundJobName(), [this]{ jobExecutingTask(); }); + getBackgroundTaskName(), [this]{ jobExecutingTask(); }); } scheduling_task->activateAndSchedule(); @@ -150,23 +169,20 @@ BackgroundJobsExecutor::BackgroundJobsExecutor( Context & global_context_) : IBackgroundJobExecutor( global_context_, - global_context_.getBackgroundProcessingTaskSleepSettings(), + global_context_.getBackgroundProcessingTaskSchedulingSettings(), {PoolConfig{PoolType::MERGE_MUTATE, global_context_.getSettingsRef().background_pool_size, CurrentMetrics::BackgroundPoolTask}}) , data(data_) { } -String BackgroundJobsExecutor::getBackgroundJobName() const +String BackgroundJobsExecutor::getBackgroundTaskName() const { return data.getStorageID().getFullTableName() + " (dataProcessingTask)"; } std::optional BackgroundJobsExecutor::getBackgroundJob() { - auto job = data.getDataProcessingJob(); - if (job) - return JobAndPool{job, PoolType::MERGE_MUTATE}; - return {}; + return data.getDataProcessingJob(); } BackgroundMovesExecutor::BackgroundMovesExecutor( @@ -174,23 +190,20 @@ BackgroundMovesExecutor::BackgroundMovesExecutor( Context & global_context_) : IBackgroundJobExecutor( global_context_, - global_context_.getBackgroundMoveTaskSleepSettings(), + global_context_.getBackgroundMoveTaskSchedulingSettings(), {PoolConfig{PoolType::MOVE, global_context_.getSettingsRef().background_move_pool_size, CurrentMetrics::BackgroundMovePoolTask}}) , data(data_) { } -String BackgroundMovesExecutor::getBackgroundJobName() const +String BackgroundMovesExecutor::getBackgroundTaskName() const { return data.getStorageID().getFullTableName() + " (dataMovingTask)"; } std::optional BackgroundMovesExecutor::getBackgroundJob() { - auto job = data.getDataMovingJob(); - if (job) - return JobAndPool{job, PoolType::MOVE}; - return {}; + return data.getDataMovingJob(); } } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index aa5eeaf2ded..e2502f534e5 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -5,87 +5,118 @@ #include #include -namespace CurrentMetrics -{ - extern const Metric BackgroundPoolTask; - extern const Metric BackgroundMovePoolTask; -} namespace DB { -struct TaskSleepSettings +/// Settings for background tasks scheduling. Each background executor has one +/// BackgroundSchedulingPoolTask and depending on execution result may put this +/// task to sleep according to settings. Look at scheduleTask function for details. +struct BackgroundTaskSchedulingSettings { double thread_sleep_seconds = 10; double thread_sleep_seconds_random_part = 1.0; double thread_sleep_seconds_if_nothing_to_do = 0.1; /// For exponential backoff. - double task_sleep_seconds_when_no_work_min = 5; - double task_sleep_seconds_when_no_work_max = 300; - double task_sleep_seconds_when_no_work_multiplier = 1.01; + double task_sleep_seconds_when_no_work_min = 6; + double task_sleep_seconds_when_no_work_max = 600; + double task_sleep_seconds_when_no_work_multiplier = 1.1; double task_sleep_seconds_when_no_work_random_part = 1.0; }; -enum PoolType +/// Pool type where we must execute new job. Each background executor can have several +/// background pools. When it receives new job it will execute new task in corresponding pool. +enum class PoolType { MERGE_MUTATE, - FETCH, MOVE, - LOW_PRIORITY, -}; - -struct PoolConfig -{ - PoolType pool_type; - size_t max_pool_size; - CurrentMetrics::Metric tasks_metric; }; +/// Result from background job providers. Function which will be executed in pool and pool type. struct JobAndPool { ThreadPool::Job job; PoolType pool_type; }; +/// Background jobs executor which execute heavy-weight background tasks for MergTree tables, like +/// background merges, moves, mutations, fetches and so on. +/// Consists of two important parts: +/// 1) Task in background scheduling pool which receives new jobs from storages and put them into required pool. +/// 2) One or more ThreadPool objects, which execute background jobs. class IBackgroundJobExecutor { +protected: Context & global_context; + + /// Configuration for single background ThreadPool + struct PoolConfig + { + /// This pool type + PoolType pool_type; + /// Max pool size in threads + size_t max_pool_size; + /// Metric that we have to increment when we execute task in this pool + CurrentMetrics::Metric tasks_metric; + }; + private: + /// Name for task in background scheduling pool String task_name; - TaskSleepSettings sleep_settings; + /// Settings for execution control of background scheduling task + BackgroundTaskSchedulingSettings sleep_settings; + /// Useful for random backoff timeouts generation pcg64 rng; + /// How many times execution of background job failed or we have + /// no new jobs. std::atomic no_work_done_count{0}; + /// Pools where we execute background jobs std::unordered_map pools; + /// Configs for background pools std::unordered_map pools_configs; + /// Scheduling task which assign jobs in background pool BackgroundSchedulePool::TaskHolder scheduling_task; + /// Mutex for thread safety std::mutex scheduling_task_mutex; public: + /// These three functions are thread safe + + /// Start background task and start to assign jobs void start(); + /// Schedule background task as soon as possible, even if it sleep at this + /// moment for some reason. void triggerTask(); + /// Finish execution: deactivate background task and wait already scheduled jobs void finish(); + /// Just call finish virtual ~IBackgroundJobExecutor(); protected: IBackgroundJobExecutor( Context & global_context_, - const TaskSleepSettings & sleep_settings_, + const BackgroundTaskSchedulingSettings & sleep_settings_, const std::vector & pools_configs_); - virtual String getBackgroundJobName() const = 0; + /// Name for task in background schedule pool + virtual String getBackgroundTaskName() const = 0; + /// Get job for background execution virtual std::optional getBackgroundJob() = 0; private: + /// Function that executes in background scheduling pool void jobExecutingTask(); + /// Recalculate timeouts when we have to check for a new job void scheduleTask(); }; +/// Main jobs executor: merges, mutations, fetches and so on class BackgroundJobsExecutor final : public IBackgroundJobExecutor { private: @@ -96,10 +127,12 @@ public: Context & global_context_); protected: - String getBackgroundJobName() const override; + String getBackgroundTaskName() const override; std::optional getBackgroundJob() override; }; +/// Move jobs executor, move parts between disks in the background +/// Does nothing in case of default configuration class BackgroundMovesExecutor final : public IBackgroundJobExecutor { private: @@ -110,7 +143,7 @@ public: Context & global_context_); protected: - String getBackgroundJobName() const override; + String getBackgroundTaskName() const override; std::optional getBackgroundJob() override; }; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 82c01acbcf4..b7aeb29f4cd 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3621,7 +3621,7 @@ bool MergeTreeData::selectPartsAndMove() return moveParts(std::move(moving_tagger)); } -ThreadPool::Job MergeTreeData::getDataMovingJob() +std::optional MergeTreeData::getDataMovingJob() { if (parts_mover.moves_blocker.isCancelled()) return {}; @@ -3630,10 +3630,10 @@ ThreadPool::Job MergeTreeData::getDataMovingJob() if (moving_tagger->parts_to_move.empty()) return {}; - return [this, moving_tagger{std::move(moving_tagger)}] () mutable + return JobAndPool{[this, moving_tagger{std::move(moving_tagger)}] () mutable { moveParts(moving_tagger); - }; + }, PoolType::MOVE}; } bool MergeTreeData::areBackgroundMovesNeeded() const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index ed0e7e33d8b..d3bfe06da99 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -40,6 +40,7 @@ class AlterCommands; class MergeTreePartsMover; class MutationCommands; class Context; +struct JobAndPool; class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; @@ -710,8 +711,10 @@ public: /// Mutex for currently_moving_parts mutable std::mutex moving_parts_mutex; - virtual ThreadPool::Job getDataProcessingJob() = 0; - ThreadPool::Job getDataMovingJob(); + /// Return main processing background job, like merge/mutate/fetch and so on + virtual std::optional getDataProcessingJob() = 0; + /// Return job to move parts between disks/volumes and so on. + std::optional getDataMovingJob(); bool areBackgroundMovesNeeded() const; protected: diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 19c8357f2b5..286ab93e6b8 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -893,7 +893,7 @@ bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_sn return true; } -ThreadPool::Job StorageMergeTree::getDataProcessingJob() +std::optional StorageMergeTree::getDataProcessingJob() { if (shutdown_called) return {}; @@ -910,17 +910,17 @@ ThreadPool::Job StorageMergeTree::getDataProcessingJob() if (merge_entry || mutate_entry) { - return [this, metadata_snapshot, merge_entry{std::move(merge_entry)}, mutate_entry{std::move(mutate_entry)}] () mutable + return JobAndPool{[this, metadata_snapshot, merge_entry{std::move(merge_entry)}, mutate_entry{std::move(mutate_entry)}] () mutable { if (merge_entry) mergeSelectedParts(metadata_snapshot, false, *merge_entry); else if (mutate_entry) mutateSelectedPart(metadata_snapshot, *mutate_entry); - }; + }, PoolType::MERGE_MUTATE}; } else if (auto lock = time_after_previous_cleanup.compareAndRestartDeferred(1)) { - return [this] () + return JobAndPool{[this] () { { auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); @@ -931,7 +931,7 @@ ThreadPool::Job StorageMergeTree::getDataProcessingJob() clearOldWriteAheadLogs(); } clearOldMutations(); - }; + }, PoolType::MERGE_MUTATE}; } return {}; } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index e6892075da8..b3d281e77b7 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -89,7 +89,7 @@ public: CheckResults checkData(const ASTPtr & query, const Context & context) override; - ThreadPool::Job getDataProcessingJob() override; + std::optional getDataProcessingJob() override; private: /// Mutex and condvar for synchronous mutations wait diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index dedaa3c0cf0..7df82620bf1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2601,8 +2601,7 @@ bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::Sel }); } - -ThreadPool::Job StorageReplicatedMergeTree::getDataProcessingJob() +std::optional StorageReplicatedMergeTree::getDataProcessingJob() { /// If replication queue is stopped exit immediately as we successfully executed the task if (queue.actions_blocker.isCancelled()) @@ -2616,10 +2615,10 @@ ThreadPool::Job StorageReplicatedMergeTree::getDataProcessingJob() if (!entry) return {}; - return [this, selected_entry{std::move(selected_entry)}] () mutable + return JobAndPool{[this, selected_entry{std::move(selected_entry)}] () mutable { processQueueEntry(selected_entry); - }; + }, PoolType::MERGE_MUTATE}; } bool StorageReplicatedMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & part) const diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index e9c54524892..79520109406 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -197,7 +197,7 @@ public: */ static void dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica, Poco::Logger * logger); - ThreadPool::Job getDataProcessingJob() override; + std::optional getDataProcessingJob() override; private: From c22c760ac1d1fa94e8fa269f8f0ae82caf0dc133 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 16 Oct 2020 13:18:56 +0300 Subject: [PATCH 042/314] Fix unit test build --- src/Storages/tests/gtest_background_executor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/tests/gtest_background_executor.cpp b/src/Storages/tests/gtest_background_executor.cpp index 77bc539b1b1..bf9a305ccc9 100644 --- a/src/Storages/tests/gtest_background_executor.cpp +++ b/src/Storages/tests/gtest_background_executor.cpp @@ -20,14 +20,14 @@ public: explicit TestJobExecutor(Context & context) :IBackgroundJobExecutor( context, - TaskSleepSettings{}, + BackgroundTaskSchedulingSettings{}, {PoolConfig{PoolType::MERGE_MUTATE, 4, CurrentMetrics::BackgroundPoolTask}}) {} protected: - String getBackgroundJobName() const override + String getBackgroundTaskName() const override { - return "TestJob"; + return "TestTask"; } std::optional getBackgroundJob() override From 518ba2ceb2acabfb28e5691b1525b4f96f1c448f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 16 Oct 2020 16:48:12 +0300 Subject: [PATCH 043/314] Safe random generator --- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 7 ++++++- src/Storages/MergeTree/BackgroundJobsExecutor.h | 4 ++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index b888eec3dfe..4efc154faa0 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -29,6 +29,11 @@ IBackgroundJobExecutor::IBackgroundJobExecutor( } } +double IBackgroundJobExecutor::getSleepRandomAdd() +{ + std::lock_guard random_lock(random_mutex); + return std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng); +} void IBackgroundJobExecutor::scheduleTask() { @@ -39,7 +44,7 @@ void IBackgroundJobExecutor::scheduleTask() auto next_time_to_execute = 1000 * (std::min( sleep_settings.task_sleep_seconds_when_no_work_max, sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) - + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + + getSleepRandomAdd()); scheduling_task->scheduleAfter(next_time_to_execute); } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index e2502f534e5..07155233ce6 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -83,6 +83,8 @@ private: BackgroundSchedulePool::TaskHolder scheduling_task; /// Mutex for thread safety std::mutex scheduling_task_mutex; + /// Mutex for pcg random generator thread safety + std::mutex random_mutex; public: /// These three functions are thread safe @@ -114,6 +116,8 @@ private: void jobExecutingTask(); /// Recalculate timeouts when we have to check for a new job void scheduleTask(); + /// Return random add for sleep in case of error + double getSleepRandomAdd(); }; /// Main jobs executor: merges, mutations, fetches and so on From 5254a5ded7bdc38446eb356570b097e771d67d89 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 18 Oct 2020 20:18:31 +0800 Subject: [PATCH 044/314] add setting aggregate_functions_null_for_empty --- src/Core/Settings.h | 3 +- src/Interpreters/TreeRewriter.cpp | 27 ++++++++++++++++++ ...gregate_functions_null_for_empty.reference | 8 ++++++ ...ing_aggregate_functions_null_for_empty.sql | 28 +++++++++++++++++++ 4 files changed, 65 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference create mode 100644 tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d73098ca6e0..ac743768ff8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -400,7 +400,8 @@ class IColumn; M(Bool, optimize_trivial_insert_select, true, "Optimize trivial 'INSERT INTO table SELECT ... FROM TABLES' query", 0) \ M(Bool, allow_experimental_database_atomic, true, "Obsolete setting, does nothing. Will be removed after 2021-02-12", 0) \ M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ - M(Bool, enable_global_with_statement, false, "Propagate WITH statements to UNION queries and all subqueries", 0) + M(Bool, enable_global_with_statement, false, "Propagate WITH statements to UNION queries and all subqueries", 0) \ + M(Bool, aggregate_functions_null_for_empty, false, "Rewrite all aggregate functions in a query, adding -OrNull suffix to them", 0) // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS below. diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 0b2f8ac3eb7..2726594f51c 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -31,6 +31,7 @@ #include #include +#include namespace DB { @@ -110,6 +111,25 @@ struct CustomizeFunctionsSuffixData char ifDistinct[] = "ifdistinct"; using CustomizeIfDistinctVisitor = InDepthNodeVisitor>, true>; +/// Used to rewrite all aggregate functions to add -OrNull suffix to them if setting `aggregate_functions_null_for_empty` is set. +struct CustomizeAggregateFunctionsSuffixData +{ + using TypeToVisit = ASTFunction; + + const String & customized_func_suffix; + + void visit(ASTFunction & func, ASTPtr &) + { + if (AggregateFunctionFactory::instance().isAggregateFunctionName(func.name) + && !endsWith(func.name, customized_func_suffix)) + { + func.name = func.name + customized_func_suffix; + } + } +}; + +using CustomizeAggregateFunctionsOrNullVisitor = InDepthNodeVisitor, true>; + /// Translate qualified names such as db.table.column, table.column, table_alias.column to names' normal form. /// Expand asterisks and qualified asterisks with column names. /// There would be columns in normal form & column aliases after translation. Column & column alias would be normalized in QueryNormalizer. @@ -692,6 +712,13 @@ void TreeRewriter::normalize(ASTPtr & query, Aliases & aliases, const Settings & CustomizeGlobalNotInVisitor(data_global_not_null_in).visit(query); } + // Rewrite all aggregate functions to add -OrNull suffix to them + if (settings.aggregate_functions_null_for_empty) + { + CustomizeAggregateFunctionsOrNullVisitor::Data data_or_null{"OrNull"}; + CustomizeAggregateFunctionsOrNullVisitor(data_or_null).visit(query); + } + /// Creates a dictionary `aliases`: alias -> ASTPtr QueryAliasesVisitor(aliases).visit(query); diff --git a/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference new file mode 100644 index 00000000000..570ea20ffad --- /dev/null +++ b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference @@ -0,0 +1,8 @@ +0 +\N +\N +\N +45 +45 +45 +45 diff --git a/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql new file mode 100644 index 00000000000..c59b592e701 --- /dev/null +++ b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql @@ -0,0 +1,28 @@ +DROP TABLE IF EXISTS defaults; + +CREATE TABLE defaults +( + n Int8 +)ENGINE = Memory(); + +SELECT sum(n) FROM defaults; +SELECT sumOrNull(n) FROM defaults; + +SET aggregate_functions_null_for_empty=1; + +SELECT sum(n) FROM defaults; +SELECT sumOrNull(n) FROM defaults; + +INSERT INTO defaults SELECT * FROM numbers(10); + +SET aggregate_functions_null_for_empty=0; + +SELECT sum(n) FROM defaults; +SELECT sumOrNull(n) FROM defaults; + +SET aggregate_functions_null_for_empty=1; + +SELECT sum(n) FROM defaults; +SELECT sumOrNull(n) FROM defaults; + +DROP TABLE defaults; From cb139abe2a0b393effa5338c959a7fb5c21d62c2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 19 Oct 2020 11:47:15 +0300 Subject: [PATCH 045/314] Better schedule after job already running --- .../MergeTree/BackgroundJobsExecutor.cpp | 46 +++++++++---------- .../MergeTree/BackgroundJobsExecutor.h | 2 +- 2 files changed, 23 insertions(+), 25 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 4efc154faa0..59633e32279 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -35,12 +35,17 @@ double IBackgroundJobExecutor::getSleepRandomAdd() return std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng); } -void IBackgroundJobExecutor::scheduleTask() +void IBackgroundJobExecutor::scheduleTask(bool job_done_or_has_job_to_do) { - auto no_work_done_times = no_work_done_count.load(std::memory_order_relaxed); - /// If we have no jobs or some errors than sleep with backoff - if (no_work_done_times != 0) + if (job_done_or_has_job_to_do) { + no_work_done_count = 0; + /// We have background jobs, schedule task as soon as possible + scheduling_task->schedule(); + } + else + { + auto no_work_done_times = no_work_done_count.fetch_add(1, std::memory_order_relaxed); auto next_time_to_execute = 1000 * (std::min( sleep_settings.task_sleep_seconds_when_no_work_max, sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) @@ -48,12 +53,6 @@ void IBackgroundJobExecutor::scheduleTask() scheduling_task->scheduleAfter(next_time_to_execute); } - else - { - /// We have background jobs, schedule task as soon as possible - scheduling_task->schedule(); - } - } namespace @@ -79,7 +78,7 @@ void IBackgroundJobExecutor::jobExecutingTask() try { auto job_and_pool = getBackgroundJob(); - if (job_and_pool) /// If we have job, than try to assign into background pool + if (job_and_pool) /// If we have job, then try to assign into background pool { auto & pool_config = pools_configs[job_and_pool->pool_type]; /// If corresponding pool is not full increment metric and assign new job @@ -94,44 +93,43 @@ try job(); /// Job done, decrement metric and reset no_work counter CurrentMetrics::values[pool_config.tasks_metric]--; - no_work_done_count = 0; + /// Job done, new empty space in pool, schedule background task + scheduleTask(true); } catch (...) { - no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; + scheduleTask(false); } - /// Job done, new empty space in pool, schedule background task - scheduleTask(); }); + /// We've scheduled task in then background pool and when it will finish we will be triggered again. But this task can be + /// extremely long and we may have a lot of other small tasks to do, so we schedule ourselfs here. + scheduleTask(true); } catch (...) { - /// With our Pool settings scheduleOrThrowOnError shouldn't throw exceptions, but for safety add catch here - no_work_done_count++; + /// With our Pool settings scheduleOrThrowOnError shouldn't throw exceptions, but for safety catch added here tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; - scheduleTask(); + scheduleTask(false); } } - else /// Pool is full and we have some work to do, let's try to schedule without backoff + else /// Pool is full and we have some work to do, let's schedule our task as fast as possible { - scheduleTask(); + scheduleTask(true); } } else /// Nothing to do, no jobs { - no_work_done_count++; - scheduleTask(); + scheduleTask(false); } } catch (...) /// Exception while we looking for a task, reschedule { - no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); - scheduleTask(); + scheduleTask(false); } void IBackgroundJobExecutor::start() diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 07155233ce6..c9f6e3649e4 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -115,7 +115,7 @@ private: /// Function that executes in background scheduling pool void jobExecutingTask(); /// Recalculate timeouts when we have to check for a new job - void scheduleTask(); + void scheduleTask(bool job_done_or_has_job_to_do); /// Return random add for sleep in case of error double getSleepRandomAdd(); }; From 514a95c292019a60188b64e72b283b8f4e16e473 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 19 Oct 2020 12:06:51 +0300 Subject: [PATCH 046/314] Fixup --- src/Storages/StorageMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 286ab93e6b8..0261f17bc55 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1464,7 +1464,7 @@ MutationCommands StorageMergeTree::getFirtsAlterMutationCommandsForPart(const Da void StorageMergeTree::startBackgroundMovesIfNeeded() { - background_executor.start(); + background_moves_executor.start(); } } From ce8ee01d754782c93e2e7104f22dfbcdeeaaefbf Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 20 Oct 2020 14:27:50 +0300 Subject: [PATCH 047/314] Fix background moves start --- src/Storages/StorageMergeTree.cpp | 6 +++--- src/Storages/StorageReplicatedMergeTree.cpp | 7 +++---- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 574fcfe933e..c0cabee42ae 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -108,8 +108,7 @@ void StorageMergeTree::startup() try { background_executor.start(); - if (areBackgroundMovesNeeded()) - background_moves_executor.start(); + startBackgroundMovesIfNeeded(); } catch (...) { @@ -1465,7 +1464,8 @@ MutationCommands StorageMergeTree::getFirtsAlterMutationCommandsForPart(const Da void StorageMergeTree::startBackgroundMovesIfNeeded() { - background_moves_executor.start(); + if (areBackgroundMovesNeeded()) + background_moves_executor.start(); } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 567bfbdb914..9ad573a4fbc 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3483,9 +3483,7 @@ void StorageReplicatedMergeTree::startup() /// If we don't separate create/start steps, race condition will happen /// between the assignment of queue_task_handle and queueTask that use the queue_task_handle. background_executor.start(); - if (areBackgroundMovesNeeded()) - background_moves_executor.start(); - + startBackgroundMovesIfNeeded(); } catch (...) { @@ -5984,7 +5982,8 @@ MutationCommands StorageReplicatedMergeTree::getFirtsAlterMutationCommandsForPar void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded() { - background_moves_executor.start(); + if (areBackgroundMovesNeeded()) + background_moves_executor.start(); } } From 7256d0157940c95559bdd47f7089addeb2c59b97 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 20 Oct 2020 16:11:01 +0300 Subject: [PATCH 048/314] Normal backoff value instead of 10 seconds --- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 6 ++++-- src/Storages/MergeTree/BackgroundJobsExecutor.h | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 59633e32279..f2061c73b9e 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -42,16 +42,18 @@ void IBackgroundJobExecutor::scheduleTask(bool job_done_or_has_job_to_do) no_work_done_count = 0; /// We have background jobs, schedule task as soon as possible scheduling_task->schedule(); + } else { auto no_work_done_times = no_work_done_count.fetch_add(1, std::memory_order_relaxed); + auto next_time_to_execute = 1000 * (std::min( sleep_settings.task_sleep_seconds_when_no_work_max, - sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) + sleep_settings.thread_sleep_seconds_if_nothing_to_do * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) + getSleepRandomAdd()); - scheduling_task->scheduleAfter(next_time_to_execute); + scheduling_task->scheduleAfter(next_time_to_execute, false); } } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index c9f6e3649e4..8660583ec02 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -19,7 +19,7 @@ struct BackgroundTaskSchedulingSettings double thread_sleep_seconds_if_nothing_to_do = 0.1; /// For exponential backoff. - double task_sleep_seconds_when_no_work_min = 6; + double task_sleep_seconds_when_no_work_min = 10; double task_sleep_seconds_when_no_work_max = 600; double task_sleep_seconds_when_no_work_multiplier = 1.1; From 30ca17b09b541fac0b53f34796c3a7729f2c6caf Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 20 Oct 2020 16:12:25 +0300 Subject: [PATCH 049/314] Comment to settings --- src/Storages/MergeTree/BackgroundJobsExecutor.h | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 8660583ec02..e7e1c982c70 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -14,16 +14,17 @@ namespace DB /// task to sleep according to settings. Look at scheduleTask function for details. struct BackgroundTaskSchedulingSettings { - double thread_sleep_seconds = 10; double thread_sleep_seconds_random_part = 1.0; double thread_sleep_seconds_if_nothing_to_do = 0.1; - - /// For exponential backoff. - double task_sleep_seconds_when_no_work_min = 10; double task_sleep_seconds_when_no_work_max = 600; + /// For exponential backoff. double task_sleep_seconds_when_no_work_multiplier = 1.1; double task_sleep_seconds_when_no_work_random_part = 1.0; + + /// deprected settings, don't affect background execution + double thread_sleep_seconds = 10; + double task_sleep_seconds_when_no_work_min = 10; }; /// Pool type where we must execute new job. Each background executor can have several From c77b154398dc2becbc15c87c6fff06eca6e4cbda Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 21 Oct 2020 00:10:55 +0300 Subject: [PATCH 050/314] Better locks in merge tree --- src/Storages/StorageMergeTree.cpp | 49 +++++++++++++------------------ src/Storages/StorageMergeTree.h | 8 ++--- 2 files changed, 25 insertions(+), 32 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c0cabee42ae..2c96c96af07 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -214,16 +214,13 @@ void StorageMergeTree::drop() dropAllData(); } -void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder & lock_holder) +void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) { - lock_holder.release(); { /// Asks to complete merges and does not allow them to start. /// This protects against "revival" of data for a removed partition after completion of merge. auto merge_blocker = stopMergesAndWait(); - /// NOTE: It's assumed that this method is called under lockForAlter. - auto parts_to_remove = getDataPartsVector(); removePartsFromWorkingSet(parts_to_remove, true); @@ -613,9 +610,9 @@ void StorageMergeTree::loadMutations() increment.value = std::max(Int64(increment.value.load()), current_mutations_by_version.rbegin()->first); } -std::optional StorageMergeTree::selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason) +std::optional StorageMergeTree::selectPartsToMerge( + const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason, TableLockHolder & /* table_lock_holder */) { - auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); std::unique_lock lock(currently_processing_in_background_mutex); auto data_settings = getSettings(); @@ -714,18 +711,18 @@ bool StorageMergeTree::merge( bool deduplicate, String * out_disable_reason) { + auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto metadata_snapshot = getInMemoryMetadataPtr(); - auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason); + auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason, table_lock_holder); if (!merge_mutate_entry) return false; - return mergeSelectedParts(metadata_snapshot, deduplicate, *merge_mutate_entry); + return mergeSelectedParts(metadata_snapshot, deduplicate, *merge_mutate_entry, table_lock_holder); } -bool StorageMergeTree::mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & merge_mutate_entry) +bool StorageMergeTree::mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & merge_mutate_entry, TableLockHolder & table_lock_holder) { - auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto & future_part = merge_mutate_entry.future_part; Stopwatch stopwatch; MutableDataPartPtr new_part; @@ -769,9 +766,8 @@ bool StorageMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & p return currently_merging_mutating_parts.count(part); } -std::optional StorageMergeTree::selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String */* disable_reason */) +std::optional StorageMergeTree::selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String */* disable_reason */, TableLockHolder & /* table_lock_holder */) { - auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); std::lock_guard lock(currently_processing_in_background_mutex); size_t max_ast_elements = global_context.getSettingsRef().max_expanded_ast_elements; @@ -850,9 +846,8 @@ std::optional StorageMergeTree::sele return {}; } -bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & merge_mutate_entry) +bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & merge_mutate_entry, TableLockHolder & table_lock_holder) { - auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto & future_part = merge_mutate_entry.future_part; auto table_id = getStorageID(); @@ -904,32 +899,30 @@ std::optional StorageMergeTree::getDataProcessingJob() auto metadata_snapshot = getInMemoryMetadataPtr(); std::optional merge_entry, mutate_entry; - merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr); + auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr, share_lock); if (!merge_entry) - mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr); + mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr, share_lock); if (merge_entry || mutate_entry) { - return JobAndPool{[this, metadata_snapshot, merge_entry{std::move(merge_entry)}, mutate_entry{std::move(mutate_entry)}] () mutable + return JobAndPool{[this, metadata_snapshot, merge_entry{std::move(merge_entry)}, mutate_entry{std::move(mutate_entry)}, share_lock] () mutable { if (merge_entry) - mergeSelectedParts(metadata_snapshot, false, *merge_entry); + mergeSelectedParts(metadata_snapshot, false, *merge_entry, share_lock); else if (mutate_entry) - mutateSelectedPart(metadata_snapshot, *mutate_entry); + mutateSelectedPart(metadata_snapshot, *mutate_entry, share_lock); }, PoolType::MERGE_MUTATE}; } else if (auto lock = time_after_previous_cleanup.compareAndRestartDeferred(1)) { - return JobAndPool{[this] () + return JobAndPool{[this, share_lock] () { - { - auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); - /// All use relative_data_path which changes during rename - /// so execute under share lock. - clearOldPartsFromFilesystem(); - clearOldTemporaryDirectories(); - clearOldWriteAheadLogs(); - } + /// All use relative_data_path which changes during rename + /// so execute under share lock. + clearOldPartsFromFilesystem(); + clearOldTemporaryDirectories(); + clearOldWriteAheadLogs(); clearOldMutations(); }, PoolType::MERGE_MUTATE}; } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index b3d281e77b7..12de2d186ab 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -150,11 +150,11 @@ private: MutationCommands commands; }; - std::optional selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * disable_reason); - bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry); + std::optional selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * disable_reason, TableLockHolder & table_lock_holder); + bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); - std::optional selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason); - bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry); + std::optional selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason, TableLockHolder & table_lock_holder); + bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); Int64 getCurrentMutationVersion( const DataPartPtr & part, From 9f47c82bbdbcd7406d6df535f3e4a2d29cc07826 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 21 Oct 2020 11:31:57 +0300 Subject: [PATCH 051/314] Sleep a little when pool is full --- .../MergeTree/BackgroundJobsExecutor.cpp | 30 ++++++++++++------- .../MergeTree/BackgroundJobsExecutor.h | 2 +- 2 files changed, 20 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index f2061c73b9e..5aba208a86e 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -35,9 +35,9 @@ double IBackgroundJobExecutor::getSleepRandomAdd() return std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng); } -void IBackgroundJobExecutor::scheduleTask(bool job_done_or_has_job_to_do) +void IBackgroundJobExecutor::scheduleTask(bool job_done, bool with_backoff) { - if (job_done_or_has_job_to_do) + if (job_done) { no_work_done_count = 0; /// We have background jobs, schedule task as soon as possible @@ -46,14 +46,22 @@ void IBackgroundJobExecutor::scheduleTask(bool job_done_or_has_job_to_do) } else { - auto no_work_done_times = no_work_done_count.fetch_add(1, std::memory_order_relaxed); + size_t next_time_to_execute; + if (with_backoff) + { + auto no_work_done_times = no_work_done_count.fetch_add(1, std::memory_order_relaxed); - auto next_time_to_execute = 1000 * (std::min( - sleep_settings.task_sleep_seconds_when_no_work_max, - sleep_settings.thread_sleep_seconds_if_nothing_to_do * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) - + getSleepRandomAdd()); + next_time_to_execute = 1000 * (std::min( + sleep_settings.task_sleep_seconds_when_no_work_max, + sleep_settings.thread_sleep_seconds_if_nothing_to_do * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) + + getSleepRandomAdd()); + } + else + { + next_time_to_execute = 1000 * sleep_settings.thread_sleep_seconds_if_nothing_to_do; + } - scheduling_task->scheduleAfter(next_time_to_execute, false); + scheduling_task->scheduleAfter(next_time_to_execute, false); } } @@ -105,7 +113,7 @@ try scheduleTask(false); } }); - /// We've scheduled task in then background pool and when it will finish we will be triggered again. But this task can be + /// We've scheduled task in the background pool and when it will finish we will be triggered again. But this task can be /// extremely long and we may have a lot of other small tasks to do, so we schedule ourselfs here. scheduleTask(true); } @@ -117,9 +125,9 @@ try scheduleTask(false); } } - else /// Pool is full and we have some work to do, let's schedule our task as fast as possible + else /// Pool is full and we have some work to do { - scheduleTask(true); + scheduleTask(false, /* with_backoff = */ false); } } else /// Nothing to do, no jobs diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index e7e1c982c70..537a6064c86 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -116,7 +116,7 @@ private: /// Function that executes in background scheduling pool void jobExecutingTask(); /// Recalculate timeouts when we have to check for a new job - void scheduleTask(bool job_done_or_has_job_to_do); + void scheduleTask(bool job_done, bool with_backoff=false); /// Return random add for sleep in case of error double getSleepRandomAdd(); }; From eb0e3a83d0b0ecb8f20fc9260ad4d8da02f22863 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Oct 2020 05:02:26 +0300 Subject: [PATCH 052/314] Support SNI in https connections to remote resources --- src/IO/HTTPCommon.cpp | 19 ++++++++++--------- src/IO/HTTPCommon.h | 7 ++++++- src/IO/ReadWriteBufferFromHTTP.cpp | 1 - 3 files changed, 16 insertions(+), 11 deletions(-) delete mode 100644 src/IO/ReadWriteBufferFromHTTP.cpp diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 6b7f30cd9b6..bda615edcd5 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -20,6 +20,7 @@ # include # include # include +# include #endif #include @@ -68,27 +69,27 @@ namespace throw Exception("Unsupported scheme in URI '" + uri.toString() + "'", ErrorCodes::UNSUPPORTED_URI_SCHEME); } - HTTPSessionPtr makeHTTPSessionImpl(const std::string & host, UInt16 port, bool https, bool keep_alive, bool resolve_host=true) + HTTPSessionPtr makeHTTPSessionImpl(const std::string & host, UInt16 port, bool https, bool keep_alive, bool resolve_host = true) { HTTPSessionPtr session; if (https) + { #if USE_SSL - session = std::make_shared(); + /// Cannot resolve host in advance, otherwise SNI won't work in Poco. + session = std::make_shared(host, port); #else throw Exception("ClickHouse was built without HTTPS support", ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME); #endif + } else - session = std::make_shared(); + { + String resolved_host = resolve_host ? DNSResolver::instance().resolveHost(host).toString() : host; + session = std::make_shared(resolved_host, port); + } ProfileEvents::increment(ProfileEvents::CreatedHTTPConnections); - if (resolve_host) - session->setHost(DNSResolver::instance().resolveHost(host).toString()); - else - session->setHost(host); - session->setPort(port); - /// doesn't work properly without patch #if defined(POCO_CLICKHOUSE_PATCH) session->setKeepAlive(keep_alive); diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index 66764b1c805..db0abe8fc6e 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -13,6 +13,7 @@ #include + namespace Poco { namespace Net @@ -24,6 +25,7 @@ namespace Net namespace DB { + constexpr int HTTP_TOO_MANY_REQUESTS = 429; class SingleEndpointHTTPSessionPool : public PoolBase @@ -39,6 +41,7 @@ private: public: SingleEndpointHTTPSessionPool(const std::string & host_, UInt16 port_, bool https_, size_t max_pool_size_); }; + using PooledHTTPSessionPtr = SingleEndpointHTTPSessionPool::Entry; using HTTPSessionPtr = std::shared_ptr; @@ -59,5 +62,7 @@ bool isRedirect(const Poco::Net::HTTPResponse::HTTPStatus status); */ std::istream * receiveResponse( Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, bool allow_redirects); -void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr, const bool allow_redirects = false); + +void assertResponseIsOk( + const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr, const bool allow_redirects = false); } diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp deleted file mode 100644 index 4d046bfe2c6..00000000000 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ /dev/null @@ -1 +0,0 @@ -#include From ec3f19b79e0b967436272fd2c753a9f5ecbe0cd3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Oct 2020 05:25:54 +0300 Subject: [PATCH 053/314] Add a test --- .../0_stateless/01532_client_sni.reference | 1 + tests/queries/0_stateless/01532_client_sni.sh | 16 ++++++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/01532_client_sni.reference create mode 100755 tests/queries/0_stateless/01532_client_sni.sh diff --git a/tests/queries/0_stateless/01532_client_sni.reference b/tests/queries/0_stateless/01532_client_sni.reference new file mode 100644 index 00000000000..7fb5e6c6321 --- /dev/null +++ b/tests/queries/0_stateless/01532_client_sni.reference @@ -0,0 +1 @@ +\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x79\x61\x6e\x64\x65\x78\x2e\x72\x75 diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh new file mode 100755 index 00000000000..9bfaf2d5427 --- /dev/null +++ b/tests/queries/0_stateless/01532_client_sni.sh @@ -0,0 +1,16 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +# Check that ClickHouse properly use SNI extension in Client Hello packet in HTTPS connection. + +strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://yandex.ru/', RawBLOB, 'data String')" 2>&1 | + grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x79\x61\x6e\x64\x65\x78\x2e\x72\x75' +# ^^^^^^^^ ^^^^^^^ ^^^^^^^ ^^ ^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ +# | | | | | +# server name data | hostname | y a n d e x . r u +# extension id len: 14 | type | +# | | +# hostnames list hostname +# len, 14 len, 9 From 237384bad99d4c27fd604398860e15af03ea74ce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Oct 2020 05:26:48 +0300 Subject: [PATCH 054/314] Fix "Arcadia" --- src/IO/ya.make | 1 - 1 file changed, 1 deletion(-) diff --git a/src/IO/ya.make b/src/IO/ya.make index 28099818b46..ad6833bc2ef 100644 --- a/src/IO/ya.make +++ b/src/IO/ya.make @@ -44,7 +44,6 @@ SRCS( ReadBufferFromPocoSocket.cpp readFloatText.cpp ReadHelpers.cpp - ReadWriteBufferFromHTTP.cpp SeekAvoidingReadBuffer.cpp UseSSL.cpp WriteBufferAIO.cpp From 608c7383e92836bded9e97db5be98fc16bf27974 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 22 Oct 2020 05:27:23 +0300 Subject: [PATCH 055/314] Update 01532_client_sni.sh --- tests/queries/0_stateless/01532_client_sni.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh index 9bfaf2d5427..51693639a37 100755 --- a/tests/queries/0_stateless/01532_client_sni.sh +++ b/tests/queries/0_stateless/01532_client_sni.sh @@ -3,7 +3,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -# Check that ClickHouse properly use SNI extension in Client Hello packet in HTTPS connection. +# Check that ClickHouse properly uses SNI extension in Client Hello packet in HTTPS connection. strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://yandex.ru/', RawBLOB, 'data String')" 2>&1 | grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x79\x61\x6e\x64\x65\x78\x2e\x72\x75' From 86de5962d2d7ff5743bc938f21c3b365405aabb3 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 22 Oct 2020 05:27:47 +0300 Subject: [PATCH 056/314] Update 01532_client_sni.sh --- tests/queries/0_stateless/01532_client_sni.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh index 51693639a37..9284661dbc0 100755 --- a/tests/queries/0_stateless/01532_client_sni.sh +++ b/tests/queries/0_stateless/01532_client_sni.sh @@ -13,4 +13,4 @@ strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * # extension id len: 14 | type | # | | # hostnames list hostname -# len, 14 len, 9 +# len, 12 len, 9 From 39e47c5338587db1dea86f4929f8caf08494ac3b Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 23 Oct 2020 11:54:00 +0300 Subject: [PATCH 057/314] Less strange lambdas --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreePartsMover.h | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 4 +- .../MergeTree/ReplicatedMergeTreeQueue.h | 17 ++- src/Storages/StorageMergeTree.cpp | 116 ++++++++---------- src/Storages/StorageMergeTree.h | 31 ++++- src/Storages/StorageReplicatedMergeTree.cpp | 16 ++- src/Storages/StorageReplicatedMergeTree.h | 4 +- 8 files changed, 106 insertions(+), 86 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b20b5e19e1a..1a7062766b0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3630,7 +3630,7 @@ std::optional MergeTreeData::getDataMovingJob() if (moving_tagger->parts_to_move.empty()) return {}; - return JobAndPool{[this, moving_tagger{std::move(moving_tagger)}] () mutable + return JobAndPool{[this, moving_tagger] () mutable { moveParts(moving_tagger); }, PoolType::MOVE}; diff --git a/src/Storages/MergeTree/MergeTreePartsMover.h b/src/Storages/MergeTree/MergeTreePartsMover.h index 332a0988d10..a1afadec7fa 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.h +++ b/src/Storages/MergeTree/MergeTreePartsMover.h @@ -16,7 +16,7 @@ namespace DB struct MergeTreeMoveEntry { std::shared_ptr part; - std::shared_ptr reserved_space; + ReservationPtr reserved_space; MergeTreeMoveEntry(const std::shared_ptr & part_, ReservationPtr reservation_) : part(part_), reserved_space(std::move(reservation_)) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index f0732774c3e..95358ecee97 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1259,7 +1259,7 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::~CurrentlyExecuting() } -ReplicatedMergeTreeQueue::SelectedEntry ReplicatedMergeTreeQueue::selectEntryToProcess(MergeTreeDataMergerMutator & merger_mutator, MergeTreeData & data) +ReplicatedMergeTreeQueue::SelectedEntryPtr ReplicatedMergeTreeQueue::selectEntryToProcess(MergeTreeDataMergerMutator & merger_mutator, MergeTreeData & data) { LogEntryPtr entry; @@ -1286,7 +1286,7 @@ ReplicatedMergeTreeQueue::SelectedEntry ReplicatedMergeTreeQueue::selectEntryToP } if (entry) - return { entry, std::shared_ptr{ new CurrentlyExecuting(entry, *this) } }; + return std::make_shared(entry, std::unique_ptr{ new CurrentlyExecuting(entry, *this) }); else return {}; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index c72569a5071..ead97579a4f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -259,6 +259,8 @@ private: ~CurrentlyExecuting(); }; + using CurrentlyExecutingPtr = std::unique_ptr; + public: ReplicatedMergeTreeQueue(StorageReplicatedMergeTree & storage_); ~ReplicatedMergeTreeQueue(); @@ -319,8 +321,19 @@ public: /** Select the next action to process. * merger_mutator is used only to check if the merges are not suspended. */ - using SelectedEntry = std::pair>; - SelectedEntry selectEntryToProcess(MergeTreeDataMergerMutator & merger_mutator, MergeTreeData & data); + struct SelectedEntry + { + ReplicatedMergeTreeQueue::LogEntryPtr log_entry; + CurrentlyExecutingPtr currently_executing_holder; + + SelectedEntry(const ReplicatedMergeTreeQueue::LogEntryPtr & log_entry_, CurrentlyExecutingPtr && currently_executing_holder_) + : log_entry(log_entry_) + , currently_executing_holder(std::move(currently_executing_holder_)) + {} + }; + + using SelectedEntryPtr = std::shared_ptr; + SelectedEntryPtr selectEntryToProcess(MergeTreeDataMergerMutator & merger_mutator, MergeTreeData & data); /** Execute `func` function to handle the action. * In this case, at runtime, mark the queue element as running diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 0e0cc13b0c7..11e8859e76c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -309,72 +309,62 @@ void StorageMergeTree::alter( /// While exists, marks parts as 'currently_merging_mutating_parts' and reserves free space on filesystem. -struct CurrentlyMergingPartsTagger +StorageMergeTree::CurrentlyMergingPartsTagger::CurrentlyMergingPartsTagger( + FutureMergedMutatedPart & future_part_, + size_t total_size, + StorageMergeTree & storage_, + const StorageMetadataPtr & metadata_snapshot, + bool is_mutation) + : future_part(future_part_), storage(storage_) { - FutureMergedMutatedPart future_part; - ReservationPtr reserved_space; + /// Assume mutex is already locked, because this method is called from mergeTask. - StorageMergeTree & storage; - -public: - CurrentlyMergingPartsTagger( - FutureMergedMutatedPart & future_part_, - size_t total_size, - StorageMergeTree & storage_, - const StorageMetadataPtr & metadata_snapshot, - bool is_mutation) - : future_part(future_part_), storage(storage_) + /// if we mutate part, than we should reserve space on the same disk, because mutations possible can create hardlinks + if (is_mutation) + reserved_space = storage.tryReserveSpace(total_size, future_part_.parts[0]->volume); + else { - /// Assume mutex is already locked, because this method is called from mergeTask. + IMergeTreeDataPart::TTLInfos ttl_infos; + size_t max_volume_index = 0; + for (auto & part_ptr : future_part_.parts) + { + ttl_infos.update(part_ptr->ttl_infos); + max_volume_index = std::max(max_volume_index, storage.getStoragePolicy()->getVolumeIndexByDisk(part_ptr->volume->getDisk())); + } - /// if we mutate part, than we should reserve space on the same disk, because mutations possible can create hardlinks + reserved_space = storage.tryReserveSpacePreferringTTLRules(metadata_snapshot, total_size, ttl_infos, time(nullptr), max_volume_index); + } + if (!reserved_space) + { if (is_mutation) - reserved_space = storage.tryReserveSpace(total_size, future_part_.parts[0]->volume); + throw Exception("Not enough space for mutating part '" + future_part_.parts[0]->name + "'", ErrorCodes::NOT_ENOUGH_SPACE); else - { - IMergeTreeDataPart::TTLInfos ttl_infos; - size_t max_volume_index = 0; - for (auto & part_ptr : future_part_.parts) - { - ttl_infos.update(part_ptr->ttl_infos); - max_volume_index = std::max(max_volume_index, storage.getStoragePolicy()->getVolumeIndexByDisk(part_ptr->volume->getDisk())); - } - - reserved_space = storage.tryReserveSpacePreferringTTLRules(metadata_snapshot, total_size, ttl_infos, time(nullptr), max_volume_index); - } - if (!reserved_space) - { - if (is_mutation) - throw Exception("Not enough space for mutating part '" + future_part_.parts[0]->name + "'", ErrorCodes::NOT_ENOUGH_SPACE); - else - throw Exception("Not enough space for merging parts", ErrorCodes::NOT_ENOUGH_SPACE); - } - - future_part_.updatePath(storage, reserved_space); - - for (const auto & part : future_part.parts) - { - if (storage.currently_merging_mutating_parts.count(part)) - throw Exception("Tagging already tagged part " + part->name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR); - } - storage.currently_merging_mutating_parts.insert(future_part.parts.begin(), future_part.parts.end()); + throw Exception("Not enough space for merging parts", ErrorCodes::NOT_ENOUGH_SPACE); } - ~CurrentlyMergingPartsTagger() + future_part_.updatePath(storage, reserved_space); + + for (const auto & part : future_part.parts) { - std::lock_guard lock(storage.currently_processing_in_background_mutex); - - for (const auto & part : future_part.parts) - { - if (!storage.currently_merging_mutating_parts.count(part)) - std::terminate(); - storage.currently_merging_mutating_parts.erase(part); - } - - storage.currently_processing_in_background_condition.notify_all(); + if (storage.currently_merging_mutating_parts.count(part)) + throw Exception("Tagging already tagged part " + part->name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR); } -}; + storage.currently_merging_mutating_parts.insert(future_part.parts.begin(), future_part.parts.end()); +} +StorageMergeTree::CurrentlyMergingPartsTagger::~CurrentlyMergingPartsTagger() +{ + std::lock_guard lock(storage.currently_processing_in_background_mutex); + + for (const auto & part : future_part.parts) + { + if (!storage.currently_merging_mutating_parts.count(part)) + std::terminate(); + storage.currently_merging_mutating_parts.erase(part); + } + + storage.currently_processing_in_background_condition.notify_all(); +} Int64 StorageMergeTree::startMutation(const MutationCommands & commands, String & mutation_file_name) { @@ -643,7 +633,7 @@ void StorageMergeTree::loadMutations() increment.value = std::max(Int64(increment.value.load()), current_mutations_by_version.rbegin()->first); } -std::optional StorageMergeTree::selectPartsToMerge( +std::shared_ptr StorageMergeTree::selectPartsToMerge( const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason, TableLockHolder & /* table_lock_holder */) { std::unique_lock lock(currently_processing_in_background_mutex); @@ -733,8 +723,8 @@ std::optional StorageMergeTree::sele return {}; } - merging_tagger = std::make_shared(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, metadata_snapshot, false); - return MergeMutateSelectedEntry{future_part, std::move(merging_tagger), {}}; + merging_tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, metadata_snapshot, false); + return std::make_shared(future_part, std::move(merging_tagger), MutationCommands{}); } bool StorageMergeTree::merge( @@ -799,7 +789,7 @@ bool StorageMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & p return currently_merging_mutating_parts.count(part); } -std::optional StorageMergeTree::selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String */* disable_reason */, TableLockHolder & /* table_lock_holder */) +std::shared_ptr StorageMergeTree::selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String */* disable_reason */, TableLockHolder & /* table_lock_holder */) { std::lock_guard lock(currently_processing_in_background_mutex); size_t max_ast_elements = global_context.getSettingsRef().max_expanded_ast_elements; @@ -873,8 +863,8 @@ std::optional StorageMergeTree::sele future_part.name = part->getNewName(new_part_info); future_part.type = part->getType(); - tagger = std::make_shared(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); - return MergeMutateSelectedEntry{future_part, std::move(tagger), commands}; + tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); + return std::make_shared(future_part, std::move(tagger), commands); } return {}; } @@ -930,7 +920,7 @@ std::optional StorageMergeTree::getDataProcessingJob() return {}; auto metadata_snapshot = getInMemoryMetadataPtr(); - std::optional merge_entry, mutate_entry; + std::shared_ptr merge_entry, mutate_entry; auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr, share_lock); @@ -939,7 +929,7 @@ std::optional StorageMergeTree::getDataProcessingJob() if (merge_entry || mutate_entry) { - return JobAndPool{[this, metadata_snapshot, merge_entry{std::move(merge_entry)}, mutate_entry{std::move(mutate_entry)}, share_lock] () mutable + return JobAndPool{[this, metadata_snapshot, merge_entry, mutate_entry, share_lock] () mutable { if (merge_entry) mergeSelectedParts(metadata_snapshot, false, *merge_entry, share_lock); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 43982ddbc78..71a32fbd203 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -21,8 +21,6 @@ namespace DB { -struct CurrentlyMergingPartsTagger; - /** See the description of the data structure in MergeTreeData. */ class StorageMergeTree final : public ext::shared_ptr_helper, public MergeTreeData @@ -140,21 +138,42 @@ private: /// Wait until mutation with version will finish mutation for all parts void waitForMutation(Int64 version, const String & file_name); - friend struct CurrentlyMergingPartsTagger; + struct CurrentlyMergingPartsTagger + { + FutureMergedMutatedPart future_part; + ReservationPtr reserved_space; - using CurrentlyMergingPartsTaggerPtr = std::shared_ptr; + StorageMergeTree & storage; + + CurrentlyMergingPartsTagger( + FutureMergedMutatedPart & future_part_, + size_t total_size, + StorageMergeTree & storage_, + const StorageMetadataPtr & metadata_snapshot, + bool is_mutation); + + ~CurrentlyMergingPartsTagger(); + }; + + using CurrentlyMergingPartsTaggerPtr = std::unique_ptr; + friend struct CurrentlyMergingPartsTagger; struct MergeMutateSelectedEntry { FutureMergedMutatedPart future_part; CurrentlyMergingPartsTaggerPtr tagger; MutationCommands commands; + MergeMutateSelectedEntry(const FutureMergedMutatedPart & future_part_, CurrentlyMergingPartsTaggerPtr && tagger_, const MutationCommands & commands_) + : future_part(future_part_) + , tagger(std::move(tagger_)) + , commands(commands_) + {} }; - std::optional selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * disable_reason, TableLockHolder & table_lock_holder); + std::shared_ptr selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * disable_reason, TableLockHolder & table_lock_holder); bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); - std::optional selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason, TableLockHolder & table_lock_holder); + std::shared_ptr selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason, TableLockHolder & table_lock_holder); bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); Int64 getCurrentMutationVersion( diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 5d670ab2edf..53f2ff14b3b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2540,10 +2540,10 @@ void StorageReplicatedMergeTree::mutationsUpdatingTask() } } -ReplicatedMergeTreeQueue::SelectedEntry StorageReplicatedMergeTree::selectQueueEntry() +ReplicatedMergeTreeQueue::SelectedEntryPtr StorageReplicatedMergeTree::selectQueueEntry() { /// This object will mark the element of the queue as running. - ReplicatedMergeTreeQueue::SelectedEntry selected; + ReplicatedMergeTreeQueue::SelectedEntryPtr selected; try { @@ -2557,10 +2557,10 @@ ReplicatedMergeTreeQueue::SelectedEntry StorageReplicatedMergeTree::selectQueueE return selected; } -bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntry & selected_entry) +bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntryPtr selected_entry) { - LogEntryPtr & entry = selected_entry.first; + LogEntryPtr & entry = selected_entry->log_entry; return queue.processEntry([this]{ return getZooKeeper(); }, entry, [&](LogEntryPtr & entry_to_process) { try @@ -2609,14 +2609,12 @@ std::optional StorageReplicatedMergeTree::getDataProcessingJob() return {}; /// This object will mark the element of the queue as running. - ReplicatedMergeTreeQueue::SelectedEntry selected_entry = selectQueueEntry(); + ReplicatedMergeTreeQueue::SelectedEntryPtr selected_entry = selectQueueEntry(); - LogEntryPtr & entry = selected_entry.first; - - if (!entry) + if (!selected_entry) return {}; - return JobAndPool{[this, selected_entry{std::move(selected_entry)}] () mutable + return JobAndPool{[this, selected_entry] () mutable { processQueueEntry(selected_entry); }, PoolType::MERGE_MUTATE}; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 277c1302540..92e17412ecc 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -418,9 +418,9 @@ private: void cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zookeeper); - ReplicatedMergeTreeQueue::SelectedEntry selectQueueEntry(); + ReplicatedMergeTreeQueue::SelectedEntryPtr selectQueueEntry(); - bool processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntry & entry); + bool processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntryPtr entry); /// Postcondition: /// either leader_election is fully initialized (node in ZK is created and the watching thread is launched) From 166b3de466a832e56863094df9f391f2a1767c05 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 23 Oct 2020 13:31:33 +0300 Subject: [PATCH 058/314] Make insert test faster --- tests/performance/insert_sequential.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/insert_sequential.xml b/tests/performance/insert_sequential.xml index 0c98c04ec8b..950a6c0c29c 100644 --- a/tests/performance/insert_sequential.xml +++ b/tests/performance/insert_sequential.xml @@ -6,6 +6,6 @@ 1 CREATE TABLE t (x UInt64) ENGINE = MergeTree() ORDER BY tuple() - INSERT INTO t SELECT * FROM numbers(20000) + INSERT INTO t SELECT * FROM numbers(3000) DROP TABLE IF EXISTS t From 84dbbdb312e5bf70aaa063799a0ebb5019b6919f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 23 Oct 2020 13:42:15 +0300 Subject: [PATCH 059/314] Rename test --- ...sequential.xml => insert_sequential_and_background_merges.xml} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/performance/{insert_sequential.xml => insert_sequential_and_background_merges.xml} (100%) diff --git a/tests/performance/insert_sequential.xml b/tests/performance/insert_sequential_and_background_merges.xml similarity index 100% rename from tests/performance/insert_sequential.xml rename to tests/performance/insert_sequential_and_background_merges.xml From d908cddb494498f4fcf27929892d999f2402792c Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Fri, 23 Oct 2020 22:08:38 +0300 Subject: [PATCH 060/314] done --- src/Interpreters/ActionsVisitor.cpp | 10 +++++++--- src/Interpreters/ActionsVisitor.h | 4 +++- src/Interpreters/ExpressionAnalyzer.cpp | 16 +++++++++++++--- src/Interpreters/ExpressionAnalyzer.h | 2 ++ src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- 5 files changed, 27 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index e0e921b003b..2f183d7dd93 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -385,7 +385,7 @@ ActionsMatcher::Data::Data( const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, const NamesAndTypesList & source_columns_, ActionsDAGPtr actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, - bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_) + bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_, bool has_having_) : context(context_) , set_size_limit(set_size_limit_) , subquery_depth(subquery_depth_) @@ -396,6 +396,7 @@ ActionsMatcher::Data::Data( , no_makeset(no_makeset_) , only_consts(only_consts_) , no_storage_or_local(no_storage_or_local_) + , has_having(has_having_) , visit_depth(0) , actions_stack(std::move(actions), context) , next_unique_suffix(actions_stack.getLastActions().getIndex().size() + 1) @@ -944,12 +945,15 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su SetPtr set = std::make_shared(data.set_size_limit, false, data.context.getSettingsRef().transform_null_in); - /** The following happens for GLOBAL INs: + /** The following happens for GLOBAL INs or INs: * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, * in the subquery_for_set object, this subquery is set as source and the temporary table _data1 as the table. * - this function shows the expression IN_data1. + * + * In case that we have HAVING with IN subquery, we have to force creating set for it. + * Also it doesn't make sence if it is GLOBAL IN or ordinary IN. */ - if (!subquery_for_set.source && data.no_storage_or_local) + if ((!subquery_for_set.source && data.no_storage_or_local) || data.has_having) { auto interpreter = interpretSubquery(right_in_operand, data.context, data.subquery_depth, {}); subquery_for_set.source = std::make_unique(); diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index f4da9932163..0179e1fd09e 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -118,6 +118,7 @@ public: bool no_makeset; bool only_consts; bool no_storage_or_local; + bool has_having; size_t visit_depth; ScopeStack actions_stack; @@ -131,7 +132,8 @@ public: Data(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, const NamesAndTypesList & source_columns_, ActionsDAGPtr actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, - bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_); + bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_, + bool has_having); /// Does result of the calculation already exists in the block. bool hasColumn(const String & column_name) const; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index f79bb36ec46..b852ab75e1f 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -396,7 +396,7 @@ void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, LogAST log; ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth, sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets, - no_subqueries, false, only_consts, !isRemoteStorage()); + no_subqueries, false, only_consts, !isRemoteStorage(), false); ActionsVisitor(visitor_data, log.stream()).visit(ast); actions = visitor_data.getActions(); } @@ -407,7 +407,17 @@ void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, bool no_sub LogAST log; ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth, sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets, - no_subqueries, true, only_consts, !isRemoteStorage()); + no_subqueries, true, only_consts, !isRemoteStorage(), false); + ActionsVisitor(visitor_data, log.stream()).visit(ast); + actions = visitor_data.getActions(); +} + +void ExpressionAnalyzer::getRootActionsHasHaving(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts) +{ + LogAST log; + ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth, + sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets, + no_subqueries, false, only_consts, !isRemoteStorage(), true); ActionsVisitor(visitor_data, log.stream()).visit(ast); actions = visitor_data.getActions(); } @@ -825,7 +835,7 @@ bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); step.required_output.push_back(select_query->having()->getColumnName()); - getRootActions(select_query->having(), only_types, step.actions()); + getRootActionsHasHaving(select_query->having(), only_types, step.actions()); return true; } diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 6389d8a142c..622c5204257 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -151,6 +151,8 @@ protected: */ void getRootActionsNoMakeSet(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts = false); + void getRootActionsHasHaving(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts = false); + /** Add aggregation keys to aggregation_keys, aggregate functions to aggregate_descriptions, * Create a set of columns aggregated_columns resulting after the aggregation, if any, * or after all the actions that are normally performed before aggregation. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d9821be4e4e..07c1942c08d 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -958,7 +958,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu preliminary_sort(); // If there is no global subqueries, we can run subqueries only when receive them on server. - if (!query_analyzer->hasGlobalSubqueries() && !subqueries_for_sets.empty()) + if (expressions.hasHaving() || (!query_analyzer->hasGlobalSubqueries() && !subqueries_for_sets.empty())) executeSubqueriesInSetsAndJoins(query_plan, subqueries_for_sets); } @@ -1071,7 +1071,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu } } - if (query_analyzer->hasGlobalSubqueries() && !subqueries_for_sets.empty()) + if (expressions.hasHaving() || (query_analyzer->hasGlobalSubqueries() && !subqueries_for_sets.empty())) executeSubqueriesInSetsAndJoins(query_plan, subqueries_for_sets); } From 7822dafcae3b19f3a4500736776c82b5a9e05dba Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Fri, 23 Oct 2020 22:37:54 +0300 Subject: [PATCH 061/314] test added --- .../01532_having_with_totals.reference | 12 +++++ .../0_stateless/01532_having_with_totals.sql | 45 +++++++++++++++++++ 2 files changed, 57 insertions(+) create mode 100644 tests/queries/0_stateless/01532_having_with_totals.reference create mode 100644 tests/queries/0_stateless/01532_having_with_totals.sql diff --git a/tests/queries/0_stateless/01532_having_with_totals.reference b/tests/queries/0_stateless/01532_having_with_totals.reference new file mode 100644 index 00000000000..2087369cae7 --- /dev/null +++ b/tests/queries/0_stateless/01532_having_with_totals.reference @@ -0,0 +1,12 @@ +127.0.0.{1,2} +1 + +0 +127.0.0.1 +1 + +0 +with explicit having +1 2 + +0 2 diff --git a/tests/queries/0_stateless/01532_having_with_totals.sql b/tests/queries/0_stateless/01532_having_with_totals.sql new file mode 100644 index 00000000000..00b8987fd83 --- /dev/null +++ b/tests/queries/0_stateless/01532_having_with_totals.sql @@ -0,0 +1,45 @@ +drop table if exists local_t; +create table local_t engine Log as select 1 a; + +SELECT '127.0.0.{1,2}'; +SELECT * +FROM +( + SELECT a + FROM remote('127.0.0.{1,2}', default, local_t) + GROUP BY a + WITH TOTALS +) +WHERE a IN +( + SELECT 1 +); + +SELECT '127.0.0.1'; +SELECT * +FROM +( + SELECT a + FROM remote('127.0.0.1', default, local_t) + GROUP BY a + WITH TOTALS +) +WHERE a IN +( + SELECT 1 +); + +SELECT 'with explicit having'; +SELECT + a, + count() +FROM remote('127.0.0.{1,2}', default, t) +GROUP BY a + WITH TOTALS +HAVING a IN +( + SELECT 1 +); + + +drop table if exists local_t; \ No newline at end of file From fe9440689a8bf9d2ef173e8e6aac1e787867ca7d Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Sat, 24 Oct 2020 00:05:24 +0300 Subject: [PATCH 062/314] better --- src/Interpreters/ActionsVisitor.h | 2 +- tests/queries/0_stateless/01532_having_with_totals.sql | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 0179e1fd09e..8c0b56f0c3c 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -132,7 +132,7 @@ public: Data(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, const NamesAndTypesList & source_columns_, ActionsDAGPtr actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, - bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_, + bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_, bool has_having); /// Does result of the calculation already exists in the block. diff --git a/tests/queries/0_stateless/01532_having_with_totals.sql b/tests/queries/0_stateless/01532_having_with_totals.sql index 00b8987fd83..10f55c8c135 100644 --- a/tests/queries/0_stateless/01532_having_with_totals.sql +++ b/tests/queries/0_stateless/01532_having_with_totals.sql @@ -6,7 +6,7 @@ SELECT * FROM ( SELECT a - FROM remote('127.0.0.{1,2}', default, local_t) + FROM remote('127.0.0.{1,2}', currentDatabase(), local_t) GROUP BY a WITH TOTALS ) @@ -20,7 +20,7 @@ SELECT * FROM ( SELECT a - FROM remote('127.0.0.1', default, local_t) + FROM remote('127.0.0.1', currentDatabase(), local_t) GROUP BY a WITH TOTALS ) @@ -33,7 +33,7 @@ SELECT 'with explicit having'; SELECT a, count() -FROM remote('127.0.0.{1,2}', default, t) +FROM remote('127.0.0.{1,2}', currentDatabase(), local_t) GROUP BY a WITH TOTALS HAVING a IN From 3f527d1d344f2a636ef327a51f639fa0c52a4ca5 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sat, 24 Oct 2020 00:41:21 +0300 Subject: [PATCH 063/314] Update order-by.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Задокументировал параметры OFFSET и FETCH. --- .../statements/select/order-by.md | 67 +++++++++++++++++++ 1 file changed, 67 insertions(+) diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index a4e5e3655c6..4be49fe18d8 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -221,3 +221,70 @@ returns │ 1970-03-12 │ 1970-01-08 │ original │ └────────────┴────────────┴──────────┘ ``` + +## OFFSET FETCH Clause {#offset-fetch} + +`OFFSET` and `FETCH` allow you to retrieve just a portion of the rows that are generated by the rest of the query. + +``` sql +OFFSET offset_row_count {ROW | ROWS} FETCH {FIRST | NEXT} fetch_row_count {ROW | ROWS} {ONLY | WITH TIES} +``` + +The `FETCH` is an alternative to the [LIMIT] (https://clickhouse.tech/docs/en/sql-reference/statements/select/limit/#limit-clause) clause and retrieves rows from a query `SELECT` type. + +The `OFFSET` says to skip that many rows before beginning to return rows. + +For example, the following query + +``` sql +SELECT * FROM test_fetch ORDER BY a OFFSET 1 ROW FETCH FIRST 3 ROWS ONLY +``` + +is identical to the query + +``` sql +SELECT * FROM test_fetch ORDER BY a LIMIT 3 OFFSET 1 +``` + +When using `FETCH`, it is important to use an [ORDER BY] (https://clickhouse.tech/docs/en/sql-reference/statements/select/order-by/#select-order-by) clause that constrains the result rows into a unique order. Otherwise, you will get an unpredictable subset of the query's rows. + +`ROW` and `ROWS` as well as `FIRST` and `NEXT` are noise words that don't influence the effects of these conditions. + +The `ONLY` option is used to return rows that immediately follow the rows omitted by the `OFFSET`. + +The `WITH TIES` option is used to return any additional rows that tie for the last place in the result set according to the `ORDER BY` clause. The `ORDER BY` is mandatory in this case. + +!!! note "Note" + According to the standard, the `OFFSET` clause must come before the `FETCH` clause if both are present. + +### Example {#example} + +Input table: + +``` text +┌─a─┬─b─┐ +│ 1 │ 1 │ +│ 2 │ 1 │ +│ 3 │ 4 │ +│ 1 │ 3 │ +│ 5 │ 4 │ +│ 0 │ 6 │ +│ 5 │ 7 │ +└───┴───┘ +``` + +Query: + +``` sql +SELECT * FROM test_fetch ORDER BY a OFFSET 1 ROW FETCH FIRST 3 ROWS ONLY +``` + +Result: + +``` text +┌─a─┬─b─┐ +│ 1 │ 1 │ +│ 1 │ 3 │ +│ 2 │ 1 │ +└───┴───┘ +``` \ No newline at end of file From c6d450f3b5c9dab5f83f4e4c39976d89106d01ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Oct 2020 09:43:29 +0300 Subject: [PATCH 064/314] Weird test --- tests/queries/0_stateless/01532_client_sni.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh index 9bfaf2d5427..a3bc7be884e 100755 --- a/tests/queries/0_stateless/01532_client_sni.sh +++ b/tests/queries/0_stateless/01532_client_sni.sh @@ -5,7 +5,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Check that ClickHouse properly use SNI extension in Client Hello packet in HTTPS connection. -strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://yandex.ru/', RawBLOB, 'data String')" 2>&1 | +sudo bash -c 'echo "127.0.0.1 yandex.ru" >> /etc/hosts' + +echo -ne 'y\r\n' | strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://yandex.ru:8443/', RawBLOB, 'data String')" 2>&1 | grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x79\x61\x6e\x64\x65\x78\x2e\x72\x75' # ^^^^^^^^ ^^^^^^^ ^^^^^^^ ^^ ^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ # | | | | | @@ -14,3 +16,5 @@ strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * # | | # hostnames list hostname # len, 14 len, 9 + +sudo bash -c 'sed -i.bak "/yandex\.ru/d" /etc/hosts' From 7187490f98a37eeb9481f5d65de88d7f5897ed40 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Oct 2020 09:45:51 +0300 Subject: [PATCH 065/314] Weird test --- tests/queries/0_stateless/01532_client_sni.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh index f9a8a131780..8a62cccf381 100755 --- a/tests/queries/0_stateless/01532_client_sni.sh +++ b/tests/queries/0_stateless/01532_client_sni.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) sudo bash -c 'echo "127.0.0.1 yandex.ru" >> /etc/hosts' -echo -ne 'y\r\n' | strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://yandex.ru:8443/', RawBLOB, 'data String')" 2>&1 | +echo -ne 'y\r\n' | strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://yandex.ru:${CLICKHOUSE_PORT_HTTPS}/', RawBLOB, 'data String')" 2>&1 | grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x79\x61\x6e\x64\x65\x78\x2e\x72\x75' # ^^^^^^^^ ^^^^^^^ ^^^^^^^ ^^ ^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ # | | | | | From b6198d2af698d01e7a7f6b27e4bbb00ac72a9349 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sat, 24 Oct 2020 13:09:53 +0300 Subject: [PATCH 066/314] Update prewhere.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправляю битые ссылки. --- docs/en/sql-reference/statements/select/prewhere.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/prewhere.md b/docs/en/sql-reference/statements/select/prewhere.md index fc43d1de0a1..2cc66e45610 100644 --- a/docs/en/sql-reference/statements/select/prewhere.md +++ b/docs/en/sql-reference/statements/select/prewhere.md @@ -2,7 +2,7 @@ toc_title: PREWHERE --- -# PREWHERE Clause {#prewhere-clause} +# PREWHERE Clause {#prewhere} Prewhere is an optimization to apply filtering more efficiently. It is enabled by default even if `PREWHERE` clause is not specified explicitly. It works by automatically moving part of [WHERE](../../../sql-reference/statements/select/where.md) condition to prewhere stage. The role of `PREWHERE` clause is only to control this optimization if you think that you know how to do it better than it happens by default. From 5c11925830ce2b0f74f5804c83aa7f666d4aa9a6 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sat, 24 Oct 2020 13:17:48 +0300 Subject: [PATCH 067/314] Update prewhere.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Вернул изменения назад. --- docs/en/sql-reference/statements/select/prewhere.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/prewhere.md b/docs/en/sql-reference/statements/select/prewhere.md index 2cc66e45610..fc43d1de0a1 100644 --- a/docs/en/sql-reference/statements/select/prewhere.md +++ b/docs/en/sql-reference/statements/select/prewhere.md @@ -2,7 +2,7 @@ toc_title: PREWHERE --- -# PREWHERE Clause {#prewhere} +# PREWHERE Clause {#prewhere-clause} Prewhere is an optimization to apply filtering more efficiently. It is enabled by default even if `PREWHERE` clause is not specified explicitly. It works by automatically moving part of [WHERE](../../../sql-reference/statements/select/where.md) condition to prewhere stage. The role of `PREWHERE` clause is only to control this optimization if you think that you know how to do it better than it happens by default. From 3da09371dc366bff81d79e4818e9cfdab239b88d Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sat, 24 Oct 2020 13:24:40 +0300 Subject: [PATCH 068/314] Update order-by.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Пытаюсь исправить битые ссылки. --- docs/en/sql-reference/statements/select/order-by.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index 4be49fe18d8..314f41e6e5f 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -230,7 +230,7 @@ returns OFFSET offset_row_count {ROW | ROWS} FETCH {FIRST | NEXT} fetch_row_count {ROW | ROWS} {ONLY | WITH TIES} ``` -The `FETCH` is an alternative to the [LIMIT] (https://clickhouse.tech/docs/en/sql-reference/statements/select/limit/#limit-clause) clause and retrieves rows from a query `SELECT` type. +The `FETCH` is an alternative to the [LIMIT] (../../../sql-reference/statements/select/limit.md) clause and retrieves rows from a query `SELECT` type. The `OFFSET` says to skip that many rows before beginning to return rows. @@ -246,7 +246,7 @@ is identical to the query SELECT * FROM test_fetch ORDER BY a LIMIT 3 OFFSET 1 ``` -When using `FETCH`, it is important to use an [ORDER BY] (https://clickhouse.tech/docs/en/sql-reference/statements/select/order-by/#select-order-by) clause that constrains the result rows into a unique order. Otherwise, you will get an unpredictable subset of the query's rows. +When using `FETCH`, it is important to use an [ORDER BY] (../../../sql-reference/statements/select/order-by.md) clause that constrains the result rows into a unique order. Otherwise, you will get an unpredictable subset of the query's rows. `ROW` and `ROWS` as well as `FIRST` and `NEXT` are noise words that don't influence the effects of these conditions. From 81f492f8b82ef73acfc1637c2edffa6d28db7998 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Oct 2020 21:50:01 +0300 Subject: [PATCH 069/314] Improve test --- tests/queries/0_stateless/01532_client_sni.reference | 2 +- tests/queries/0_stateless/01532_client_sni.sh | 10 +++------- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01532_client_sni.reference b/tests/queries/0_stateless/01532_client_sni.reference index 7fb5e6c6321..4bf62e12856 100644 --- a/tests/queries/0_stateless/01532_client_sni.reference +++ b/tests/queries/0_stateless/01532_client_sni.reference @@ -1 +1 @@ -\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x79\x61\x6e\x64\x65\x78\x2e\x72\x75 +\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x6c\x6f\x63\x61\x6c\x68\x6f\x73\x74 diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh index 8a62cccf381..92c2f6be233 100755 --- a/tests/queries/0_stateless/01532_client_sni.sh +++ b/tests/queries/0_stateless/01532_client_sni.sh @@ -5,16 +5,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Check that ClickHouse properly uses SNI extension in Client Hello packet in HTTPS connection. -sudo bash -c 'echo "127.0.0.1 yandex.ru" >> /etc/hosts' - -echo -ne 'y\r\n' | strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://yandex.ru:${CLICKHOUSE_PORT_HTTPS}/', RawBLOB, 'data String')" 2>&1 | - grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x79\x61\x6e\x64\x65\x78\x2e\x72\x75' +echo -ne 'y\r\n' | strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://localhost:${CLICKHOUSE_PORT_HTTPS}/', RawBLOB, 'data String')" 2>&1 | + grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x6c\x6f\x63\x61\x6c\x68\x6f\x73\x74' # ^^^^^^^^ ^^^^^^^ ^^^^^^^ ^^ ^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ # | | | | | -# server name data | hostname | y a n d e x . r u +# server name data | hostname | l o c a l h o s t # extension id len: 14 | type | # | | # hostnames list hostname # len, 12 len, 9 - -sudo bash -c 'sed -i.bak "/yandex\.ru/d" /etc/hosts' From fb5d281c69d3e546545e2fcaaca431f98f0cb2a8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Oct 2020 05:00:30 +0300 Subject: [PATCH 070/314] Better test --- tests/queries/0_stateless/01532_client_sni.reference | 2 +- tests/queries/0_stateless/01532_client_sni.sh | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01532_client_sni.reference b/tests/queries/0_stateless/01532_client_sni.reference index 4bf62e12856..879b5791c7b 100644 --- a/tests/queries/0_stateless/01532_client_sni.reference +++ b/tests/queries/0_stateless/01532_client_sni.reference @@ -1 +1 @@ -\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x6c\x6f\x63\x61\x6c\x68\x6f\x73\x74 +0000000e000c0000096c6f63616c686f7374 diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh index 92c2f6be233..b3f2a12bf22 100755 --- a/tests/queries/0_stateless/01532_client_sni.sh +++ b/tests/queries/0_stateless/01532_client_sni.sh @@ -5,8 +5,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Check that ClickHouse properly uses SNI extension in Client Hello packet in HTTPS connection. -echo -ne 'y\r\n' | strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://localhost:${CLICKHOUSE_PORT_HTTPS}/', RawBLOB, 'data String')" 2>&1 | - grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x6c\x6f\x63\x61\x6c\x68\x6f\x73\x74' +nc -q0 -l 5678 | xxd -p | grep -oF $'0000000e000c0000096c6f63616c686f7374' & + +${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://localhost:5678/', RawBLOB, 'data String')" 2>&1 | grep -v -F 'Timeout' + +# grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x6c\x6f\x63\x61\x6c\x68\x6f\x73\x74' # ^^^^^^^^ ^^^^^^^ ^^^^^^^ ^^ ^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ # | | | | | # server name data | hostname | l o c a l h o s t From 7a65ef8c6704f5862ecfa9efa453bd5338572a1b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Oct 2020 05:01:04 +0300 Subject: [PATCH 071/314] Better test --- tests/queries/0_stateless/01532_client_sni.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh index b3f2a12bf22..0b122713fde 100755 --- a/tests/queries/0_stateless/01532_client_sni.sh +++ b/tests/queries/0_stateless/01532_client_sni.sh @@ -17,3 +17,5 @@ ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://localhost:5678/', RawBLO # | | # hostnames list hostname # len, 12 len, 9 + +wait From ce016ba8c930707f9a53ed712fb786e213a8045f Mon Sep 17 00:00:00 2001 From: ana-uvarova Date: Tue, 27 Oct 2020 10:22:14 +0300 Subject: [PATCH 072/314] Draft --- docs/en/operations/system-tables/asynchronous_metric_log.md | 2 +- docs/en/operations/system-tables/metric_log.md | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/asynchronous_metric_log.md b/docs/en/operations/system-tables/asynchronous_metric_log.md index 75607cc30b0..e1eabdefa2f 100644 --- a/docs/en/operations/system-tables/asynchronous_metric_log.md +++ b/docs/en/operations/system-tables/asynchronous_metric_log.md @@ -1,6 +1,6 @@ ## system.asynchronous_metric_log {#system-tables-async-log} -Contains the historical values for `system.asynchronous_metrics`, which are saved once per minute. This feature is enabled by default. +Contains the historical values for `system.asynchronous_metrics`, which are saved once per minute. Enabled by default. Columns: diff --git a/docs/en/operations/system-tables/metric_log.md b/docs/en/operations/system-tables/metric_log.md index f1910407949..c6aefc5034f 100644 --- a/docs/en/operations/system-tables/metric_log.md +++ b/docs/en/operations/system-tables/metric_log.md @@ -1,6 +1,7 @@ # system.metric_log {#system_tables-metric_log} Contains history of metrics values from tables `system.metrics` and `system.events`, periodically flushed to disk. + To turn on metrics history collection on `system.metric_log`, create `/etc/clickhouse-server/config.d/metric_log.xml` with following content: ``` xml @@ -14,6 +15,11 @@ To turn on metrics history collection on `system.metric_log`, create `/etc/click ``` +Columns: +- `event_date` ([Date](../../sql-reference/data-types/date.md)) — Event date. +- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Event time. +- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Event time with microseconds resolution. + **Example** ``` sql From 2e0a979e3a2aaae76a4714ef120d4319752fa86a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 29 Oct 2020 06:39:43 +0300 Subject: [PATCH 073/314] Fix inconsistency in FormatFactory --- programs/client/Client.cpp | 2 + programs/copier/ClusterCopierApp.cpp | 2 + programs/local/LocalServer.cpp | 2 + programs/odbc-bridge/ODBCBridge.cpp | 4 + programs/server/Server.cpp | 2 + src/Formats/FormatFactory.cpp | 144 ------------------------ src/Formats/FormatFactory.h | 3 - src/Formats/registerFormats.cpp | 160 +++++++++++++++++++++++++++ src/Formats/registerFormats.h | 9 ++ src/Formats/ya.make | 1 + 10 files changed, 182 insertions(+), 147 deletions(-) create mode 100644 src/Formats/registerFormats.cpp create mode 100644 src/Formats/registerFormats.h diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 808a505b5e4..4ead4b3b41a 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -75,6 +75,7 @@ #include #include #include +#include #include #include #include @@ -461,6 +462,7 @@ private: { UseSSL use_ssl; + registerFormats(); registerFunctions(); registerAggregateFunctions(); diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index 08a7e50a9d7..c2946e12c34 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -1,6 +1,7 @@ #include "ClusterCopierApp.h" #include #include +#include #include @@ -122,6 +123,7 @@ void ClusterCopierApp::mainImpl() registerStorages(); registerDictionaries(); registerDisks(); + registerFormats(); static const std::string default_database = "_local"; DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database, *context)); diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index b9dde555788..9ecc2a50a42 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -33,6 +33,7 @@ #include #include #include +#include #include #include #include @@ -224,6 +225,7 @@ try registerStorages(); registerDictionaries(); registerDisks(); + registerFormats(); /// Maybe useless if (config().has("macros")) diff --git a/programs/odbc-bridge/ODBCBridge.cpp b/programs/odbc-bridge/ODBCBridge.cpp index 3f119fbf7ba..24aa8e32ddb 100644 --- a/programs/odbc-bridge/ODBCBridge.cpp +++ b/programs/odbc-bridge/ODBCBridge.cpp @@ -18,11 +18,13 @@ #include #include #include +#include #include #include #include #include + namespace DB { namespace ErrorCodes @@ -160,6 +162,8 @@ int ODBCBridge::main(const std::vector & /*args*/) if (is_help) return Application::EXIT_OK; + registerFormats(); + LOG_INFO(log, "Starting up"); Poco::Net::ServerSocket socket; auto address = socketBindListen(socket, hostname, port, log); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b85cb5e75f2..ed18793a537 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -51,6 +51,7 @@ #include #include #include +#include #include #include #include @@ -266,6 +267,7 @@ int Server::main(const std::vector & /*args*/) registerStorages(); registerDictionaries(); registerDisks(); + registerFormats(); CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 4dc5b816420..9e04d717949 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -333,150 +333,6 @@ void FormatFactory::registerFileSegmentationEngine(const String & name, FileSegm target = std::move(file_segmentation_engine); } -/// File Segmentation Engines for parallel reading - -void registerFileSegmentationEngineTabSeparated(FormatFactory & factory); -void registerFileSegmentationEngineCSV(FormatFactory & factory); -void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); -void registerFileSegmentationEngineRegexp(FormatFactory & factory); -void registerFileSegmentationEngineJSONAsString(FormatFactory & factory); -void registerFileSegmentationEngineLineAsString(FormatFactory & factory); - -/// Formats for both input/output. - -void registerInputFormatNative(FormatFactory & factory); -void registerOutputFormatNative(FormatFactory & factory); - -void registerInputFormatProcessorNative(FormatFactory & factory); -void registerOutputFormatProcessorNative(FormatFactory & factory); -void registerInputFormatProcessorRowBinary(FormatFactory & factory); -void registerOutputFormatProcessorRowBinary(FormatFactory & factory); -void registerInputFormatProcessorTabSeparated(FormatFactory & factory); -void registerOutputFormatProcessorTabSeparated(FormatFactory & factory); -void registerInputFormatProcessorValues(FormatFactory & factory); -void registerOutputFormatProcessorValues(FormatFactory & factory); -void registerInputFormatProcessorCSV(FormatFactory & factory); -void registerOutputFormatProcessorCSV(FormatFactory & factory); -void registerInputFormatProcessorTSKV(FormatFactory & factory); -void registerOutputFormatProcessorTSKV(FormatFactory & factory); -void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); -void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); -void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory); -void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory); -void registerInputFormatProcessorProtobuf(FormatFactory & factory); -void registerOutputFormatProcessorProtobuf(FormatFactory & factory); -void registerInputFormatProcessorTemplate(FormatFactory & factory); -void registerOutputFormatProcessorTemplate(FormatFactory & factory); -void registerInputFormatProcessorMsgPack(FormatFactory & factory); -void registerOutputFormatProcessorMsgPack(FormatFactory & factory); -void registerInputFormatProcessorORC(FormatFactory & factory); -void registerOutputFormatProcessorORC(FormatFactory & factory); -void registerInputFormatProcessorParquet(FormatFactory & factory); -void registerOutputFormatProcessorParquet(FormatFactory & factory); -void registerInputFormatProcessorArrow(FormatFactory & factory); -void registerOutputFormatProcessorArrow(FormatFactory & factory); -void registerInputFormatProcessorAvro(FormatFactory & factory); -void registerOutputFormatProcessorAvro(FormatFactory & factory); -void registerInputFormatProcessorRawBLOB(FormatFactory & factory); -void registerOutputFormatProcessorRawBLOB(FormatFactory & factory); - -/// Output only (presentational) formats. - -void registerOutputFormatNull(FormatFactory & factory); - -void registerOutputFormatProcessorPretty(FormatFactory & factory); -void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory); -void registerOutputFormatProcessorPrettySpace(FormatFactory & factory); -void registerOutputFormatProcessorVertical(FormatFactory & factory); -void registerOutputFormatProcessorJSON(FormatFactory & factory); -void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); -void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); -void registerOutputFormatProcessorXML(FormatFactory & factory); -void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); -void registerOutputFormatProcessorNull(FormatFactory & factory); -void registerOutputFormatProcessorMySQLWire(FormatFactory & factory); -void registerOutputFormatProcessorMarkdown(FormatFactory & factory); -void registerOutputFormatProcessorPostgreSQLWire(FormatFactory & factory); - -/// Input only formats. - -void registerInputFormatProcessorRegexp(FormatFactory & factory); -void registerInputFormatProcessorJSONAsString(FormatFactory & factory); -void registerInputFormatProcessorLineAsString(FormatFactory & factory); -void registerInputFormatProcessorCapnProto(FormatFactory & factory); - -FormatFactory::FormatFactory() -{ - registerFileSegmentationEngineTabSeparated(*this); - registerFileSegmentationEngineCSV(*this); - registerFileSegmentationEngineJSONEachRow(*this); - registerFileSegmentationEngineRegexp(*this); - registerFileSegmentationEngineJSONAsString(*this); - registerFileSegmentationEngineLineAsString(*this); - - registerInputFormatNative(*this); - registerOutputFormatNative(*this); - - registerInputFormatProcessorNative(*this); - registerOutputFormatProcessorNative(*this); - registerInputFormatProcessorRowBinary(*this); - registerOutputFormatProcessorRowBinary(*this); - registerInputFormatProcessorTabSeparated(*this); - registerOutputFormatProcessorTabSeparated(*this); - registerInputFormatProcessorValues(*this); - registerOutputFormatProcessorValues(*this); - registerInputFormatProcessorCSV(*this); - registerOutputFormatProcessorCSV(*this); - registerInputFormatProcessorTSKV(*this); - registerOutputFormatProcessorTSKV(*this); - registerInputFormatProcessorJSONEachRow(*this); - registerOutputFormatProcessorJSONEachRow(*this); - registerInputFormatProcessorJSONCompactEachRow(*this); - registerOutputFormatProcessorJSONCompactEachRow(*this); - registerInputFormatProcessorProtobuf(*this); - registerOutputFormatProcessorProtobuf(*this); - registerInputFormatProcessorTemplate(*this); - registerOutputFormatProcessorTemplate(*this); - registerInputFormatProcessorMsgPack(*this); - registerOutputFormatProcessorMsgPack(*this); - registerInputFormatProcessorRawBLOB(*this); - registerOutputFormatProcessorRawBLOB(*this); - -#if !defined(ARCADIA_BUILD) - registerInputFormatProcessorORC(*this); - registerOutputFormatProcessorORC(*this); - registerInputFormatProcessorParquet(*this); - registerOutputFormatProcessorParquet(*this); - registerInputFormatProcessorArrow(*this); - registerOutputFormatProcessorArrow(*this); - registerInputFormatProcessorAvro(*this); - registerOutputFormatProcessorAvro(*this); -#endif - - registerOutputFormatNull(*this); - - registerOutputFormatProcessorPretty(*this); - registerOutputFormatProcessorPrettyCompact(*this); - registerOutputFormatProcessorPrettySpace(*this); - registerOutputFormatProcessorVertical(*this); - registerOutputFormatProcessorJSON(*this); - registerOutputFormatProcessorJSONCompact(*this); - registerOutputFormatProcessorJSONEachRowWithProgress(*this); - registerOutputFormatProcessorXML(*this); - registerOutputFormatProcessorODBCDriver2(*this); - registerOutputFormatProcessorNull(*this); - registerOutputFormatProcessorMySQLWire(*this); - registerOutputFormatProcessorMarkdown(*this); - registerOutputFormatProcessorPostgreSQLWire(*this); - - registerInputFormatProcessorRegexp(*this); - registerInputFormatProcessorJSONAsString(*this); - registerInputFormatProcessorLineAsString(*this); - -#if !defined(ARCADIA_BUILD) - registerInputFormatProcessorCapnProto(*this); -#endif -} FormatFactory & FormatFactory::instance() { diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index d49414e3944..dbf6a3d65b2 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -96,7 +96,6 @@ private: using FormatsDictionary = std::unordered_map; public: - static FormatFactory & instance(); BlockInputStreamPtr getInput( @@ -137,8 +136,6 @@ public: private: FormatsDictionary dict; - FormatFactory(); - const Creators & getCreators(const String & name) const; }; diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp new file mode 100644 index 00000000000..96b2c4ee384 --- /dev/null +++ b/src/Formats/registerFormats.cpp @@ -0,0 +1,160 @@ +#if !defined(ARCADIA_BUILD) +# include +#endif + +#include + + +namespace DB +{ + +/// File Segmentation Engines for parallel reading + +void registerFileSegmentationEngineTabSeparated(FormatFactory & factory); +void registerFileSegmentationEngineCSV(FormatFactory & factory); +void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); +void registerFileSegmentationEngineRegexp(FormatFactory & factory); +void registerFileSegmentationEngineJSONAsString(FormatFactory & factory); +void registerFileSegmentationEngineLineAsString(FormatFactory & factory); + +/// Formats for both input/output. + +void registerInputFormatNative(FormatFactory & factory); +void registerOutputFormatNative(FormatFactory & factory); + +void registerInputFormatProcessorNative(FormatFactory & factory); +void registerOutputFormatProcessorNative(FormatFactory & factory); +void registerInputFormatProcessorRowBinary(FormatFactory & factory); +void registerOutputFormatProcessorRowBinary(FormatFactory & factory); +void registerInputFormatProcessorTabSeparated(FormatFactory & factory); +void registerOutputFormatProcessorTabSeparated(FormatFactory & factory); +void registerInputFormatProcessorValues(FormatFactory & factory); +void registerOutputFormatProcessorValues(FormatFactory & factory); +void registerInputFormatProcessorCSV(FormatFactory & factory); +void registerOutputFormatProcessorCSV(FormatFactory & factory); +void registerInputFormatProcessorTSKV(FormatFactory & factory); +void registerOutputFormatProcessorTSKV(FormatFactory & factory); +void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); +void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); +void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory); +void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory); +void registerInputFormatProcessorProtobuf(FormatFactory & factory); +void registerOutputFormatProcessorProtobuf(FormatFactory & factory); +void registerInputFormatProcessorTemplate(FormatFactory & factory); +void registerOutputFormatProcessorTemplate(FormatFactory & factory); +void registerInputFormatProcessorMsgPack(FormatFactory & factory); +void registerOutputFormatProcessorMsgPack(FormatFactory & factory); +void registerInputFormatProcessorORC(FormatFactory & factory); +void registerOutputFormatProcessorORC(FormatFactory & factory); +void registerInputFormatProcessorParquet(FormatFactory & factory); +void registerOutputFormatProcessorParquet(FormatFactory & factory); +void registerInputFormatProcessorArrow(FormatFactory & factory); +void registerOutputFormatProcessorArrow(FormatFactory & factory); +void registerInputFormatProcessorAvro(FormatFactory & factory); +void registerOutputFormatProcessorAvro(FormatFactory & factory); +void registerInputFormatProcessorRawBLOB(FormatFactory & factory); +void registerOutputFormatProcessorRawBLOB(FormatFactory & factory); + +/// Output only (presentational) formats. + +void registerOutputFormatNull(FormatFactory & factory); + +void registerOutputFormatProcessorPretty(FormatFactory & factory); +void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory); +void registerOutputFormatProcessorPrettySpace(FormatFactory & factory); +void registerOutputFormatProcessorVertical(FormatFactory & factory); +void registerOutputFormatProcessorJSON(FormatFactory & factory); +void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); +void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); +void registerOutputFormatProcessorXML(FormatFactory & factory); +void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); +void registerOutputFormatProcessorNull(FormatFactory & factory); +void registerOutputFormatProcessorMySQLWire(FormatFactory & factory); +void registerOutputFormatProcessorMarkdown(FormatFactory & factory); +void registerOutputFormatProcessorPostgreSQLWire(FormatFactory & factory); + +/// Input only formats. + +void registerInputFormatProcessorRegexp(FormatFactory & factory); +void registerInputFormatProcessorJSONAsString(FormatFactory & factory); +void registerInputFormatProcessorLineAsString(FormatFactory & factory); +void registerInputFormatProcessorCapnProto(FormatFactory & factory); + + +void registerFormats() +{ + auto & factory = FormatFactory::instance(); + + registerFileSegmentationEngineTabSeparated(factory); + registerFileSegmentationEngineCSV(factory); + registerFileSegmentationEngineJSONEachRow(factory); + registerFileSegmentationEngineRegexp(factory); + registerFileSegmentationEngineJSONAsString(factory); + registerFileSegmentationEngineLineAsString(factory); + + registerInputFormatNative(factory); + registerOutputFormatNative(factory); + + registerInputFormatProcessorNative(factory); + registerOutputFormatProcessorNative(factory); + registerInputFormatProcessorRowBinary(factory); + registerOutputFormatProcessorRowBinary(factory); + registerInputFormatProcessorTabSeparated(factory); + registerOutputFormatProcessorTabSeparated(factory); + registerInputFormatProcessorValues(factory); + registerOutputFormatProcessorValues(factory); + registerInputFormatProcessorCSV(factory); + registerOutputFormatProcessorCSV(factory); + registerInputFormatProcessorTSKV(factory); + registerOutputFormatProcessorTSKV(factory); + registerInputFormatProcessorJSONEachRow(factory); + registerOutputFormatProcessorJSONEachRow(factory); + registerInputFormatProcessorJSONCompactEachRow(factory); + registerOutputFormatProcessorJSONCompactEachRow(factory); + registerInputFormatProcessorProtobuf(factory); + registerOutputFormatProcessorProtobuf(factory); + registerInputFormatProcessorTemplate(factory); + registerOutputFormatProcessorTemplate(factory); + registerInputFormatProcessorMsgPack(factory); + registerOutputFormatProcessorMsgPack(factory); + registerInputFormatProcessorRawBLOB(factory); + registerOutputFormatProcessorRawBLOB(factory); + +#if !defined(ARCADIA_BUILD) + registerInputFormatProcessorORC(factory); + registerOutputFormatProcessorORC(factory); + registerInputFormatProcessorParquet(factory); + registerOutputFormatProcessorParquet(factory); + registerInputFormatProcessorArrow(factory); + registerOutputFormatProcessorArrow(factory); + registerInputFormatProcessorAvro(factory); + registerOutputFormatProcessorAvro(factory); +#endif + + registerOutputFormatNull(factory); + + registerOutputFormatProcessorPretty(factory); + registerOutputFormatProcessorPrettyCompact(factory); + registerOutputFormatProcessorPrettySpace(factory); + registerOutputFormatProcessorVertical(factory); + registerOutputFormatProcessorJSON(factory); + registerOutputFormatProcessorJSONCompact(factory); + registerOutputFormatProcessorJSONEachRowWithProgress(factory); + registerOutputFormatProcessorXML(factory); + registerOutputFormatProcessorODBCDriver2(factory); + registerOutputFormatProcessorNull(factory); + registerOutputFormatProcessorMySQLWire(factory); + registerOutputFormatProcessorMarkdown(factory); + registerOutputFormatProcessorPostgreSQLWire(factory); + + registerInputFormatProcessorRegexp(factory); + registerInputFormatProcessorJSONAsString(factory); + registerInputFormatProcessorLineAsString(factory); + +#if !defined(ARCADIA_BUILD) + registerInputFormatProcessorCapnProto(factory); +#endif +} + +} + diff --git a/src/Formats/registerFormats.h b/src/Formats/registerFormats.h new file mode 100644 index 00000000000..e4ff79248d0 --- /dev/null +++ b/src/Formats/registerFormats.h @@ -0,0 +1,9 @@ +#pragma once + +namespace DB +{ + +void registerFormats(); + +} + diff --git a/src/Formats/ya.make b/src/Formats/ya.make index b4f7b073e21..fb6c5cf8cf1 100644 --- a/src/Formats/ya.make +++ b/src/Formats/ya.make @@ -23,6 +23,7 @@ SRCS( ProtobufReader.cpp ProtobufSchemas.cpp ProtobufWriter.cpp + registerFormats.cpp verbosePrintString.cpp ) From a7b5db912c44a95478fd1a3f6b7d52a0c2ff5f58 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 29 Oct 2020 06:42:49 +0300 Subject: [PATCH 074/314] odbc-bridge: two libraries less --- programs/odbc-bridge/CMakeLists.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/programs/odbc-bridge/CMakeLists.txt b/programs/odbc-bridge/CMakeLists.txt index 8cfa110adad..ae60bcd5f0b 100644 --- a/programs/odbc-bridge/CMakeLists.txt +++ b/programs/odbc-bridge/CMakeLists.txt @@ -13,8 +13,6 @@ set (CLICKHOUSE_ODBC_BRIDGE_SOURCES ) set (CLICKHOUSE_ODBC_BRIDGE_LINK PRIVATE - clickhouse_parsers - clickhouse_aggregate_functions daemon dbms Poco::Data From 0440958c3ecb47066328264a0bf67a11a6198da2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 29 Oct 2020 07:27:12 +0300 Subject: [PATCH 075/314] Add missing modification --- programs/obfuscator/Obfuscator.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 756aab0a574..b326d550fec 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -1047,6 +1048,8 @@ public: int mainEntryClickHouseObfuscator(int argc, char ** argv) try { + registerFormats(); + using namespace DB; namespace po = boost::program_options; From f2bdfcef41f6c2ff261f7040e128a8d7dedea80c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 29 Oct 2020 08:18:42 +0300 Subject: [PATCH 076/314] Add missing modification --- programs/obfuscator/Obfuscator.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index b326d550fec..d0d7f201c68 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -1048,11 +1048,11 @@ public: int mainEntryClickHouseObfuscator(int argc, char ** argv) try { - registerFormats(); - using namespace DB; namespace po = boost::program_options; + registerFormats(); + po::options_description description = createOptionsDescription("Options", getTerminalWidth()); description.add_options() ("help", "produce help message") From 1520b6aa2aa5833637ea45fab55ce3e0b57caf60 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 29 Oct 2020 08:27:55 +0300 Subject: [PATCH 077/314] Take out odbc-bridge from clickhouse bundle --- programs/CMakeLists.txt | 3 --- programs/odbc-bridge/CMakeLists.txt | 23 ++++++++++------------- programs/odbc-bridge/tests/CMakeLists.txt | 3 +-- 3 files changed, 11 insertions(+), 18 deletions(-) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 3577ee3df31..3817bc62bcb 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -227,9 +227,6 @@ else () install (FILES ${CMAKE_CURRENT_BINARY_DIR}/clickhouse-git-import DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) list(APPEND CLICKHOUSE_BUNDLE clickhouse-git-import) endif () - if(ENABLE_CLICKHOUSE_ODBC_BRIDGE) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-odbc-bridge) - endif() install (TARGETS clickhouse RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) diff --git a/programs/odbc-bridge/CMakeLists.txt b/programs/odbc-bridge/CMakeLists.txt index ae60bcd5f0b..043c0d5dd7d 100644 --- a/programs/odbc-bridge/CMakeLists.txt +++ b/programs/odbc-bridge/CMakeLists.txt @@ -10,17 +10,8 @@ set (CLICKHOUSE_ODBC_BRIDGE_SOURCES PingHandler.cpp SchemaAllowedHandler.cpp validateODBCConnectionString.cpp + odbc-bridge.cpp ) -set (CLICKHOUSE_ODBC_BRIDGE_LINK - PRIVATE - daemon - dbms - Poco::Data - PUBLIC - Poco::Data::ODBC -) - -clickhouse_program_add_library(odbc-bridge) if (OS_LINUX) # clickhouse-odbc-bridge is always a separate binary. @@ -28,10 +19,16 @@ if (OS_LINUX) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") endif () -add_executable(clickhouse-odbc-bridge odbc-bridge.cpp) -set_target_properties(clickhouse-odbc-bridge PROPERTIES RUNTIME_OUTPUT_DIRECTORY ..) +add_executable(clickhouse-odbc-bridge ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}) -clickhouse_program_link_split_binary(odbc-bridge) +target_link_libraries(clickhouse-odbc-bridge PRIVATE + daemon + clickhouse_common_io + Poco::Data + Poco::Data::ODBC +) + +set_target_properties(clickhouse-odbc-bridge PROPERTIES RUNTIME_OUTPUT_DIRECTORY ..) if (USE_GDB_ADD_INDEX) add_custom_command(TARGET clickhouse-odbc-bridge POST_BUILD COMMAND ${GDB_ADD_INDEX_EXE} ../clickhouse-odbc-bridge COMMENT "Adding .gdb-index to clickhouse-odbc-bridge" VERBATIM) diff --git a/programs/odbc-bridge/tests/CMakeLists.txt b/programs/odbc-bridge/tests/CMakeLists.txt index 60e7afab969..3e0af8c940f 100644 --- a/programs/odbc-bridge/tests/CMakeLists.txt +++ b/programs/odbc-bridge/tests/CMakeLists.txt @@ -1,3 +1,2 @@ -add_executable (validate-odbc-connection-string validate-odbc-connection-string.cpp) -clickhouse_target_link_split_lib(validate-odbc-connection-string odbc-bridge) +add_executable (validate-odbc-connection-string validate-odbc-connection-string.cpp ../validateODBCConnectionString.cpp) target_link_libraries (validate-odbc-connection-string PRIVATE clickhouse_common_io) From 3fb8d08dd41bab1d05def74749486cabcc746994 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 12 Oct 2020 21:22:09 +0300 Subject: [PATCH 078/314] Add errorCodeToName() function This patch adds errorCode() function that will return macro name by the exception code, useful for analyzing query_log and similar. Later some descriptions can be added. v2: replace hana::map/fusion::vector with external script v3: - use binary cmake dir for .sh - use jump table over suboptimal std::unordered_map (with very poor hash function for int -- std::hash) - cleanup errorCodeToName (drop extra templates and headers) - rename errorCode -> errorCodeToName - fix arcadia build (by not providing those helpers there) - fix build on CI, by using CMAKE_CXX_COMPILER for parsing the file --- .../functions/other-functions.md | 20 ++++++ src/Functions/.gitignore | 1 + src/Functions/CMakeLists.txt | 16 +++++ src/Functions/errorCodeToName.cpp | 66 +++++++++++++++++++ src/Functions/errorCodes.sh | 61 +++++++++++++++++ .../registerFunctionsMiscellaneous.cpp | 6 ++ src/Functions/ya.make.in | 2 +- .../01544_errorCodeToName.reference | 3 + .../0_stateless/01544_errorCodeToName.sql | 3 + 9 files changed, 177 insertions(+), 1 deletion(-) create mode 100644 src/Functions/.gitignore create mode 100644 src/Functions/errorCodeToName.cpp create mode 100755 src/Functions/errorCodes.sh create mode 100644 tests/queries/0_stateless/01544_errorCodeToName.reference create mode 100644 tests/queries/0_stateless/01544_errorCodeToName.sql diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 2cc80dcffc1..dd5df19d40f 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -1657,4 +1657,24 @@ Result: 10 10 19 19 39 39 ``` +## errorCodeToName {#error-code-to-name} + +**Returned value** + +- Variable name for the error code. + +Type: [String](../../sql-reference/data-types/string.md). + +**Syntax** + +``` sql +errorCodeToName(1) +``` + +Result: + +``` text +UNSUPPORTED_METHOD +``` + [Original article](https://clickhouse.tech/docs/en/query_language/functions/other_functions/) diff --git a/src/Functions/.gitignore b/src/Functions/.gitignore new file mode 100644 index 00000000000..25db3625c77 --- /dev/null +++ b/src/Functions/.gitignore @@ -0,0 +1 @@ +/errorCodes.generated.cpp diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index bdf89c983f1..d949ab8283f 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -6,6 +6,20 @@ add_headers_and_sources(clickhouse_functions .) list(REMOVE_ITEM clickhouse_functions_sources IFunctionImpl.cpp FunctionFactory.cpp FunctionHelpers.cpp) list(REMOVE_ITEM clickhouse_functions_headers IFunctionImpl.h FunctionFactory.h FunctionHelpers.h) +set(ERROR_CODES_IN ${CMAKE_CURRENT_SOURCE_DIR}/../Common/ErrorCodes.cpp) +set(ERROR_CODES_OUT ${CMAKE_CURRENT_BINARY_DIR}/errorCodes.generated.cpp) +add_custom_target(generate-error-codes + env + ERROR_CODES_IN_FILE=${ERROR_CODES_IN} + ERROR_CODES_OUT_FILE=${ERROR_CODES_OUT} + CXX=${CMAKE_CXX_COMPILER} + ${CMAKE_CURRENT_SOURCE_DIR}/errorCodes.sh + SOURCES errorCodes.sh + WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR} + BYPRODUCTS ${ERROR_CODES_OUT} +) +list(APPEND clickhouse_functions_sources ${ERROR_CODES_OUT}) + if (NOT USE_H3) list (REMOVE_ITEM clickhouse_functions_sources geoToH3.cpp @@ -53,6 +67,8 @@ endif() target_include_directories(clickhouse_functions SYSTEM PRIVATE ${SPARSEHASH_INCLUDE_DIR}) +add_dependencies(clickhouse_functions generate-error-codes) + if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL") diff --git a/src/Functions/errorCodeToName.cpp b/src/Functions/errorCodeToName.cpp new file mode 100644 index 00000000000..6621a6f20bb --- /dev/null +++ b/src/Functions/errorCodeToName.cpp @@ -0,0 +1,66 @@ +#if !defined(ARCADIA_BUILD) + +#include +#include +#include +#include +#include + +extern std::string_view errorCodeToName(int code); + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +/** errorCodeToName() - returns the variable name for the error code. + */ +class FunctionErrorCodeToName : public IFunction +{ +public: + static constexpr auto name = "errorCodeToName"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & types) const override + { + if (!isNumber(types.at(0))) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The argument of function {} must have simple numeric type, possibly Nullable", name); + + return std::make_shared(); + } + + ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + auto & input_column = *arguments[0].column; + auto col_res = ColumnString::create(); + + for (size_t i = 0; i < input_rows_count; ++i) + { + const Int64 error_code = input_column.getInt(i); + std::string_view error_name = errorCodeToName(error_code); + col_res->insertData(error_name.data(), error_name.size()); + } + + return col_res; + } +}; + + +void registerFunctionErrorCodeToName(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + +#endif diff --git a/src/Functions/errorCodes.sh b/src/Functions/errorCodes.sh new file mode 100755 index 00000000000..711465be614 --- /dev/null +++ b/src/Functions/errorCodes.sh @@ -0,0 +1,61 @@ +#!/usr/bin/env bash + +# Parse src/Common/ErrorCodes.cpp +# And generate src/Functions/errorCodes.generated.cpp +# For errorCode() function. +# +# Later it may contain some description of the error. + +set -e +set -o pipefail + +CUR_DIR="$(readlink -f "$(dirname "${BASH_SOURCE[0]}")")" +ERROR_CODES_IN_FILE=${ERROR_CODES_IN_FILE=$CUR_DIR/../Common/ErrorCodes.cpp} +ERROR_CODES_OUT_FILE=${ERROR_CODES_OUT_FILE=$CUR_DIR/errorCodes.generated.cpp} +CXX=${CXX=g++} + +trap 'rm -f $TMP_FILE' EXIT +TMP_FILE="$(mktemp clichouse_generate_errorCodes_XXXXXXXX.cpp)" + +function parse_for_errorCodeToName() +{ + # This is the simplest command that can be written to parse the file + # And it does not requires any extra tools and works everywhere where you have g++/clang++ + $CXX -E "$ERROR_CODES_IN_FILE" | { + awk -F '[ =;]*' '/extern const int / { printf(" case %s: return std::string_view(\"%s\");\n", $(NF-1), $(NF-2)); }' + } +} + +function generate_errorCodeToName() +{ + cat < + +std::string_view errorCodeToName(int code) +{ + switch (code) + { + case 0: return std::string_view("OK"); +$(parse_for_errorCodeToName) + default: return std::string_view(""); + } +}; + +EOL +} + +function main() +{ + generate_errorCodeToName > "$TMP_FILE" + + if [[ ! -e $ERROR_CODES_OUT_FILE ]]; then + cp -a "$TMP_FILE" "$ERROR_CODES_OUT_FILE" + fi + # update it only if it differs, to avoid costly recompilation + if ! diff -q "$TMP_FILE" "$ERROR_CODES_OUT_FILE"; then + cp -a "$TMP_FILE" "$ERROR_CODES_OUT_FILE" + fi +} +main "$@" diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 414f6ec5f8e..4a989831b48 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -64,6 +64,9 @@ void registerFunctionCountDigits(FunctionFactory &); void registerFunctionGlobalVariable(FunctionFactory &); void registerFunctionHasThreadFuzzer(FunctionFactory &); void registerFunctionInitializeAggregation(FunctionFactory &); +#if !defined(ARCADIA_BUILD) +void registerFunctionErrorCodeToName(FunctionFactory &); +#endif #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -128,6 +131,9 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionGlobalVariable(factory); registerFunctionHasThreadFuzzer(factory); registerFunctionInitializeAggregation(factory); +#if !defined(ARCADIA_BUILD) + registerFunctionErrorCodeToName(factory); +#endif #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/src/Functions/ya.make.in b/src/Functions/ya.make.in index 2a66aa5553e..e70658bb79d 100644 --- a/src/Functions/ya.make.in +++ b/src/Functions/ya.make.in @@ -34,7 +34,7 @@ PEERDIR( CFLAGS(-g0) SRCS( - + ) END() diff --git a/tests/queries/0_stateless/01544_errorCodeToName.reference b/tests/queries/0_stateless/01544_errorCodeToName.reference new file mode 100644 index 00000000000..4f0073384d9 --- /dev/null +++ b/tests/queries/0_stateless/01544_errorCodeToName.reference @@ -0,0 +1,3 @@ + +OK +UNSUPPORTED_METHOD diff --git a/tests/queries/0_stateless/01544_errorCodeToName.sql b/tests/queries/0_stateless/01544_errorCodeToName.sql new file mode 100644 index 00000000000..07b46767b73 --- /dev/null +++ b/tests/queries/0_stateless/01544_errorCodeToName.sql @@ -0,0 +1,3 @@ +SELECT errorCodeToName(toUInt32(-1)); +SELECT errorCodeToName(0); +SELECT errorCodeToName(1); From b2e232289507b18ce09861d9cc339d7094606fd4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Oct 2020 01:00:04 +0300 Subject: [PATCH 079/314] Add system.errors table MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Contains error codes with number of times they have been triggered. Columns: - `name` ([String](../../sql-reference/data-types/string.md)) — name of the error (`errorCodeToName`). - `code` ([Int32](../../sql-reference/data-types/int-uint.md)) — code number of the error. - `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) - number of times this error has been happened. **Example** ``` sql SELECT * FROM system.errors WHERE value > 0 ORDER BY code ASC LIMIT 1 ┌─name─────────────┬─code─┬─value─┐ │ CANNOT_OPEN_FILE │ 76 │ 1 │ └──────────────────┴──────┴───────┘ --- docker/test/fasttest/run.sh | 2 + docs/en/operations/system-tables/errors.md | 23 ++++++++++ src/CMakeLists.txt | 15 +++++++ src/Common/Exception.cpp | 6 +++ src/{Functions => Common}/errorCodes.sh | 44 ++++++++++++++++--- src/Common/ya.make.in | 2 +- src/Functions/CMakeLists.txt | 14 ------ src/Storages/System/StorageSystemErrors.cpp | 43 ++++++++++++++++++ src/Storages/System/StorageSystemErrors.h | 29 ++++++++++++ src/Storages/System/attachSystemTables.cpp | 2 + src/Storages/ya.make | 1 + .../0_stateless/01545_system_errors.reference | 1 + .../0_stateless/01545_system_errors.sh | 9 ++++ 13 files changed, 170 insertions(+), 21 deletions(-) create mode 100644 docs/en/operations/system-tables/errors.md rename src/{Functions => Common}/errorCodes.sh (58%) create mode 100644 src/Storages/System/StorageSystemErrors.cpp create mode 100644 src/Storages/System/StorageSystemErrors.h create mode 100644 tests/queries/0_stateless/01545_system_errors.reference create mode 100755 tests/queries/0_stateless/01545_system_errors.sh diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 8300c31681e..bcba50e9d81 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -277,6 +277,8 @@ TESTS_TO_SKIP=( # Require python libraries like scipy, pandas and numpy 01322_ttest_scipy + + 01545_system_errors ) time clickhouse-test -j 8 --order=random --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/test_log.txt" diff --git a/docs/en/operations/system-tables/errors.md b/docs/en/operations/system-tables/errors.md new file mode 100644 index 00000000000..53e8a397217 --- /dev/null +++ b/docs/en/operations/system-tables/errors.md @@ -0,0 +1,23 @@ +# system.errors {#system_tables-errors} + +Contains error codes with number of times they have been triggered. + +Columns: + +- `name` ([String](../../sql-reference/data-types/string.md)) — name of the error (`errorCodeToName`). +- `code` ([Int32](../../sql-reference/data-types/int-uint.md)) — code number of the error. +- `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) - number of times this error has been happened. + +**Example** + +``` sql +SELECT * +FROM system.errors +WHERE value > 0 +ORDER BY code ASC +LIMIT 1 + +┌─name─────────────┬─code─┬─value─┐ +│ CANNOT_OPEN_FILE │ 76 │ 1 │ +└──────────────────┴──────┴───────┘ +``` diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 085269847e4..86c78d9309a 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -86,6 +86,21 @@ endif() list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON}) +# ErrorCodes +set(ERROR_CODES_IN ${CMAKE_CURRENT_SOURCE_DIR}/Common/ErrorCodes.cpp) +set(ERROR_CODES_OUT ${CMAKE_CURRENT_BINARY_DIR}/Common/errorCodes.generated.cpp) +add_custom_target(generate-error-codes + env + ERROR_CODES_IN_FILE=${ERROR_CODES_IN} + ERROR_CODES_OUT_FILE=${ERROR_CODES_OUT} + CXX=${CMAKE_CXX_COMPILER} + ${CMAKE_CURRENT_SOURCE_DIR}/Common/errorCodes.sh + SOURCES Common/errorCodes.sh + WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR} + BYPRODUCTS ${ERROR_CODES_OUT} +) +list(APPEND clickhouse_common_io_sources ${ERROR_CODES_OUT}) + list (APPEND dbms_sources Functions/IFunction.cpp Functions/FunctionFactory.cpp Functions/FunctionHelpers.cpp Functions/extractTimeZoneFromFunctionArguments.cpp) list (APPEND dbms_headers Functions/IFunctionImpl.h Functions/FunctionFactory.h Functions/FunctionHelpers.h Functions/extractTimeZoneFromFunctionArguments.h) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 716938eb3d6..4b9bb595ce6 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -12,14 +12,19 @@ #include #include #include +#include #include #include +#include #include +#include #if !defined(ARCADIA_BUILD) # include #endif +extern HashMap, DefaultHash> error_codes_count; + namespace DB { @@ -45,6 +50,7 @@ Exception::Exception(const std::string & msg, int code) LOG_FATAL(&Poco::Logger::root(), "Logical error: '{}'.", msg); abort(); } + ++error_codes_count[code]; #endif } diff --git a/src/Functions/errorCodes.sh b/src/Common/errorCodes.sh similarity index 58% rename from src/Functions/errorCodes.sh rename to src/Common/errorCodes.sh index 711465be614..3a4520e2dad 100755 --- a/src/Functions/errorCodes.sh +++ b/src/Common/errorCodes.sh @@ -17,20 +17,26 @@ CXX=${CXX=g++} trap 'rm -f $TMP_FILE' EXIT TMP_FILE="$(mktemp clichouse_generate_errorCodes_XXXXXXXX.cpp)" -function parse_for_errorCodeToName() +function parse_ErrorCodes() { # This is the simplest command that can be written to parse the file # And it does not requires any extra tools and works everywhere where you have g++/clang++ + # + # Generate: + # + # CODE VAR_NAME + # $CXX -E "$ERROR_CODES_IN_FILE" | { - awk -F '[ =;]*' '/extern const int / { printf(" case %s: return std::string_view(\"%s\");\n", $(NF-1), $(NF-2)); }' + awk -F '[ =;]*' '/extern const int / { print $(NF-1), $(NF-2); }' } } - +function parse_for_errorCodeToName() +{ + parse_ErrorCodes | awk '{ printf(" case %s: return std::string_view(\"%s\");\n", $1, $2); }' +} function generate_errorCodeToName() { cat < std::string_view errorCodeToName(int code) @@ -46,9 +52,35 @@ $(parse_for_errorCodeToName) EOL } +function parse_for_error_codes_count() +{ + parse_ErrorCodes | awk '{ printf(" error_codes_count[%s] = 0; /* %s */\n", $1, $2); }' +} +function generate_error_codes_count() +{ + cat < +#include +#include +#include + +HashMap, DefaultHash> error_codes_count; + +struct InitializeErrorCodesCount +{ + InitializeErrorCodesCount() + { +$(parse_for_error_codes_count) + } +} error_codes_count_initialize; +EOL +} + function main() { - generate_errorCodeToName > "$TMP_FILE" + echo "// autogenerated by ${BASH_SOURCE[0]}" > "$TMP_FILE" + generate_errorCodeToName >> "$TMP_FILE" + generate_error_codes_count >> "$TMP_FILE" if [[ ! -e $ERROR_CODES_OUT_FILE ]]; then cp -a "$TMP_FILE" "$ERROR_CODES_OUT_FILE" diff --git a/src/Common/ya.make.in b/src/Common/ya.make.in index f8b7601e215..e0e3fd9a944 100644 --- a/src/Common/ya.make.in +++ b/src/Common/ya.make.in @@ -23,7 +23,7 @@ INCLUDE(${ARCADIA_ROOT}/clickhouse/cmake/yandex/ya.make.versions.inc) CFLAGS(-g0) SRCS( - + ) END() diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index d949ab8283f..b528b3a276f 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -6,20 +6,6 @@ add_headers_and_sources(clickhouse_functions .) list(REMOVE_ITEM clickhouse_functions_sources IFunctionImpl.cpp FunctionFactory.cpp FunctionHelpers.cpp) list(REMOVE_ITEM clickhouse_functions_headers IFunctionImpl.h FunctionFactory.h FunctionHelpers.h) -set(ERROR_CODES_IN ${CMAKE_CURRENT_SOURCE_DIR}/../Common/ErrorCodes.cpp) -set(ERROR_CODES_OUT ${CMAKE_CURRENT_BINARY_DIR}/errorCodes.generated.cpp) -add_custom_target(generate-error-codes - env - ERROR_CODES_IN_FILE=${ERROR_CODES_IN} - ERROR_CODES_OUT_FILE=${ERROR_CODES_OUT} - CXX=${CMAKE_CXX_COMPILER} - ${CMAKE_CURRENT_SOURCE_DIR}/errorCodes.sh - SOURCES errorCodes.sh - WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR} - BYPRODUCTS ${ERROR_CODES_OUT} -) -list(APPEND clickhouse_functions_sources ${ERROR_CODES_OUT}) - if (NOT USE_H3) list (REMOVE_ITEM clickhouse_functions_sources geoToH3.cpp diff --git a/src/Storages/System/StorageSystemErrors.cpp b/src/Storages/System/StorageSystemErrors.cpp new file mode 100644 index 00000000000..5ab6d02a78f --- /dev/null +++ b/src/Storages/System/StorageSystemErrors.cpp @@ -0,0 +1,43 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +extern std::string_view errorCodeToName(int code); +extern HashMap, DefaultHash> error_codes_count; + +namespace DB +{ + +NamesAndTypesList StorageSystemErrors::getNamesAndTypes() +{ + return { + { "name", std::make_shared() }, + { "code", std::make_shared() }, + { "value", std::make_shared() }, + }; +} + + +void StorageSystemErrors::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const +{ + for (const auto & error_code_pair : error_codes_count) + { + size_t col_num = 0; + res_columns[col_num++]->insert(errorCodeToName(error_code_pair.getKey())); + res_columns[col_num++]->insert(error_code_pair.getKey()); + res_columns[col_num++]->insert(uint64_t(error_code_pair.getMapped())); + } +} + +} diff --git a/src/Storages/System/StorageSystemErrors.h b/src/Storages/System/StorageSystemErrors.h new file mode 100644 index 00000000000..3f2d3020bcd --- /dev/null +++ b/src/Storages/System/StorageSystemErrors.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class Context; + + +/** Implements the `distribution_queue` system table, which allows you to view the INSERT queues for the Distributed tables. + */ +class StorageSystemErrors final : public ext::shared_ptr_helper, public IStorageSystemOneBlock +{ + friend struct ext::shared_ptr_helper; +public: + std::string getName() const override { return "SystemErrors"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 2b7ee363f05..1ea24c1c4ba 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -41,6 +41,7 @@ #include #include #endif +#include #include #include #include @@ -110,6 +111,7 @@ void attachSystemTablesLocal(IDatabase & system_database) #if !defined(ARCADIA_BUILD) attach(system_database, "licenses"); attach(system_database, "time_zones"); + attach(system_database, "errors"); #endif #ifdef OS_LINUX attach(system_database, "stack_trace"); diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 107433b5e73..a196b2f4fda 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -154,6 +154,7 @@ SRCS( System/StorageSystemDisks.cpp System/StorageSystemDistributionQueue.cpp System/StorageSystemEnabledRoles.cpp + System/StorageSystemErrors.cpp System/StorageSystemEvents.cpp System/StorageSystemFormats.cpp System/StorageSystemFunctions.cpp diff --git a/tests/queries/0_stateless/01545_system_errors.reference b/tests/queries/0_stateless/01545_system_errors.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01545_system_errors.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01545_system_errors.sh b/tests/queries/0_stateless/01545_system_errors.sh new file mode 100755 index 00000000000..402c4e34116 --- /dev/null +++ b/tests/queries/0_stateless/01545_system_errors.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +prev="$(${CLICKHOUSE_CLIENT} -q "SELECT value FROM system.errors WHERE name = 'FUNCTION_THROW_IF_VALUE_IS_NON_ZERO'")" +$CLICKHOUSE_CLIENT -q 'SELECT throwIf(1)' >& /dev/null +cur="$(${CLICKHOUSE_CLIENT} -q "SELECT value FROM system.errors WHERE name = 'FUNCTION_THROW_IF_VALUE_IS_NON_ZERO'")" +echo $((cur - prev)) From a81b0418acdfdfce2c4a9078dfe71808e430a38f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Oct 2020 09:42:08 +0300 Subject: [PATCH 080/314] Make errorCodeToName() return LowCardinality(String) --- docs/en/sql-reference/functions/other-functions.md | 2 +- src/Functions/errorCodeToName.cpp | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index dd5df19d40f..31ed47c3195 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -1663,7 +1663,7 @@ Result: - Variable name for the error code. -Type: [String](../../sql-reference/data-types/string.md). +Type: [LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md). **Syntax** diff --git a/src/Functions/errorCodeToName.cpp b/src/Functions/errorCodeToName.cpp index 6621a6f20bb..e8f34c5edd0 100644 --- a/src/Functions/errorCodeToName.cpp +++ b/src/Functions/errorCodeToName.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -36,13 +37,13 @@ public: if (!isNumber(types.at(0))) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The argument of function {} must have simple numeric type, possibly Nullable", name); - return std::make_shared(); + return std::make_shared(std::make_shared()); } - ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr & res_type, size_t input_rows_count) const override { auto & input_column = *arguments[0].column; - auto col_res = ColumnString::create(); + auto col_res = res_type->createColumn(); for (size_t i = 0; i < input_rows_count; ++i) { From b7eac807f59f74036adf164b261952acfb47a53b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Oct 2020 10:35:16 +0300 Subject: [PATCH 081/314] Fix readability-qualified-auto in errorCodeToName() --- src/Functions/errorCodeToName.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/errorCodeToName.cpp b/src/Functions/errorCodeToName.cpp index e8f34c5edd0..8c9f49b2ac8 100644 --- a/src/Functions/errorCodeToName.cpp +++ b/src/Functions/errorCodeToName.cpp @@ -42,7 +42,7 @@ public: ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr & res_type, size_t input_rows_count) const override { - auto & input_column = *arguments[0].column; + const auto & input_column = *arguments[0].column; auto col_res = res_type->createColumn(); for (size_t i = 0; i < input_rows_count; ++i) From b42f77a791f74b1a2ffd5cd84fb7bf7cbc78c8d9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Oct 2020 10:07:42 +0300 Subject: [PATCH 082/314] Covert error codes list to lookup table and drop generation of them. --- src/CMakeLists.txt | 15 - src/Common/ErrorCodes.cpp | 1069 +++++++++-------- src/Common/ErrorCodes.h | 41 + src/Common/Exception.cpp | 8 +- src/Common/errorCodes.sh | 93 -- src/Functions/CMakeLists.txt | 2 - src/Functions/errorCodeToName.cpp | 6 +- src/Storages/System/StorageSystemErrors.cpp | 37 +- .../01544_errorCodeToName.reference | 1 + .../0_stateless/01544_errorCodeToName.sql | 1 + 10 files changed, 618 insertions(+), 655 deletions(-) create mode 100644 src/Common/ErrorCodes.h delete mode 100755 src/Common/errorCodes.sh diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 86c78d9309a..085269847e4 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -86,21 +86,6 @@ endif() list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON}) -# ErrorCodes -set(ERROR_CODES_IN ${CMAKE_CURRENT_SOURCE_DIR}/Common/ErrorCodes.cpp) -set(ERROR_CODES_OUT ${CMAKE_CURRENT_BINARY_DIR}/Common/errorCodes.generated.cpp) -add_custom_target(generate-error-codes - env - ERROR_CODES_IN_FILE=${ERROR_CODES_IN} - ERROR_CODES_OUT_FILE=${ERROR_CODES_OUT} - CXX=${CMAKE_CXX_COMPILER} - ${CMAKE_CURRENT_SOURCE_DIR}/Common/errorCodes.sh - SOURCES Common/errorCodes.sh - WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR} - BYPRODUCTS ${ERROR_CODES_OUT} -) -list(APPEND clickhouse_common_io_sources ${ERROR_CODES_OUT}) - list (APPEND dbms_sources Functions/IFunction.cpp Functions/FunctionFactory.cpp Functions/FunctionHelpers.cpp Functions/extractTimeZoneFromFunctionArguments.cpp) list (APPEND dbms_headers Functions/IFunctionImpl.h Functions/FunctionFactory.h Functions/FunctionHelpers.h Functions/extractTimeZoneFromFunctionArguments.h) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index b14c090c848..7b671427753 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -1,526 +1,565 @@ +#include + +/** Previously, these constants were located in one enum. + * But in this case there is a problem: when you add a new constant, you need to recompile + * all translation units that use at least one constant (almost the whole project). + * Therefore it is made so that definitions of constants are located here, in one file, + * and their declaration are in different files, at the place of use. + * + * Later it was converted to the lookup table, to provide: + * - errorCodeToName() + * - system.errors table + */ + +#define APPLY_FOR_ERROR_CODES(M) \ + M(OK, 0) \ + M(UNSUPPORTED_METHOD, 1) \ + M(UNSUPPORTED_PARAMETER, 2) \ + M(UNEXPECTED_END_OF_FILE, 3) \ + M(EXPECTED_END_OF_FILE, 4) \ + M(CANNOT_PARSE_TEXT, 6) \ + M(INCORRECT_NUMBER_OF_COLUMNS, 7) \ + M(THERE_IS_NO_COLUMN, 8) \ + M(SIZES_OF_COLUMNS_DOESNT_MATCH, 9) \ + M(NOT_FOUND_COLUMN_IN_BLOCK, 10) \ + M(POSITION_OUT_OF_BOUND, 11) \ + M(PARAMETER_OUT_OF_BOUND, 12) \ + M(SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH, 13) \ + M(DUPLICATE_COLUMN, 15) \ + M(NO_SUCH_COLUMN_IN_TABLE, 16) \ + M(DELIMITER_IN_STRING_LITERAL_DOESNT_MATCH, 17) \ + M(CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN, 18) \ + M(SIZE_OF_FIXED_STRING_DOESNT_MATCH, 19) \ + M(NUMBER_OF_COLUMNS_DOESNT_MATCH, 20) \ + M(CANNOT_READ_ALL_DATA_FROM_TAB_SEPARATED_INPUT, 21) \ + M(CANNOT_PARSE_ALL_VALUE_FROM_TAB_SEPARATED_INPUT, 22) \ + M(CANNOT_READ_FROM_ISTREAM, 23) \ + M(CANNOT_WRITE_TO_OSTREAM, 24) \ + M(CANNOT_PARSE_ESCAPE_SEQUENCE, 25) \ + M(CANNOT_PARSE_QUOTED_STRING, 26) \ + M(CANNOT_PARSE_INPUT_ASSERTION_FAILED, 27) \ + M(CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, 28) \ + M(CANNOT_PRINT_INTEGER, 29) \ + M(CANNOT_READ_SIZE_OF_COMPRESSED_CHUNK, 30) \ + M(CANNOT_READ_COMPRESSED_CHUNK, 31) \ + M(ATTEMPT_TO_READ_AFTER_EOF, 32) \ + M(CANNOT_READ_ALL_DATA, 33) \ + M(TOO_MANY_ARGUMENTS_FOR_FUNCTION, 34) \ + M(TOO_FEW_ARGUMENTS_FOR_FUNCTION, 35) \ + M(BAD_ARGUMENTS, 36) \ + M(UNKNOWN_ELEMENT_IN_AST, 37) \ + M(CANNOT_PARSE_DATE, 38) \ + M(TOO_LARGE_SIZE_COMPRESSED, 39) \ + M(CHECKSUM_DOESNT_MATCH, 40) \ + M(CANNOT_PARSE_DATETIME, 41) \ + M(NUMBER_OF_ARGUMENTS_DOESNT_MATCH, 42) \ + M(ILLEGAL_TYPE_OF_ARGUMENT, 43) \ + M(ILLEGAL_COLUMN, 44) \ + M(ILLEGAL_NUMBER_OF_RESULT_COLUMNS, 45) \ + M(UNKNOWN_FUNCTION, 46) \ + M(UNKNOWN_IDENTIFIER, 47) \ + M(NOT_IMPLEMENTED, 48) \ + M(LOGICAL_ERROR, 49) \ + M(UNKNOWN_TYPE, 50) \ + M(EMPTY_LIST_OF_COLUMNS_QUERIED, 51) \ + M(COLUMN_QUERIED_MORE_THAN_ONCE, 52) \ + M(TYPE_MISMATCH, 53) \ + M(STORAGE_DOESNT_ALLOW_PARAMETERS, 54) \ + M(STORAGE_REQUIRES_PARAMETER, 55) \ + M(UNKNOWN_STORAGE, 56) \ + M(TABLE_ALREADY_EXISTS, 57) \ + M(TABLE_METADATA_ALREADY_EXISTS, 58) \ + M(ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, 59) \ + M(UNKNOWN_TABLE, 60) \ + M(ONLY_FILTER_COLUMN_IN_BLOCK, 61) \ + M(SYNTAX_ERROR, 62) \ + M(UNKNOWN_AGGREGATE_FUNCTION, 63) \ + M(CANNOT_READ_AGGREGATE_FUNCTION_FROM_TEXT, 64) \ + M(CANNOT_WRITE_AGGREGATE_FUNCTION_AS_TEXT, 65) \ + M(NOT_A_COLUMN, 66) \ + M(ILLEGAL_KEY_OF_AGGREGATION, 67) \ + M(CANNOT_GET_SIZE_OF_FIELD, 68) \ + M(ARGUMENT_OUT_OF_BOUND, 69) \ + M(CANNOT_CONVERT_TYPE, 70) \ + M(CANNOT_WRITE_AFTER_END_OF_BUFFER, 71) \ + M(CANNOT_PARSE_NUMBER, 72) \ + M(UNKNOWN_FORMAT, 73) \ + M(CANNOT_READ_FROM_FILE_DESCRIPTOR, 74) \ + M(CANNOT_WRITE_TO_FILE_DESCRIPTOR, 75) \ + M(CANNOT_OPEN_FILE, 76) \ + M(CANNOT_CLOSE_FILE, 77) \ + M(UNKNOWN_TYPE_OF_QUERY, 78) \ + M(INCORRECT_FILE_NAME, 79) \ + M(INCORRECT_QUERY, 80) \ + M(UNKNOWN_DATABASE, 81) \ + M(DATABASE_ALREADY_EXISTS, 82) \ + M(DIRECTORY_DOESNT_EXIST, 83) \ + M(DIRECTORY_ALREADY_EXISTS, 84) \ + M(FORMAT_IS_NOT_SUITABLE_FOR_INPUT, 85) \ + M(RECEIVED_ERROR_FROM_REMOTE_IO_SERVER, 86) \ + M(CANNOT_SEEK_THROUGH_FILE, 87) \ + M(CANNOT_TRUNCATE_FILE, 88) \ + M(UNKNOWN_COMPRESSION_METHOD, 89) \ + M(EMPTY_LIST_OF_COLUMNS_PASSED, 90) \ + M(SIZES_OF_MARKS_FILES_ARE_INCONSISTENT, 91) \ + M(EMPTY_DATA_PASSED, 92) \ + M(UNKNOWN_AGGREGATED_DATA_VARIANT, 93) \ + M(CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS, 94) \ + M(CANNOT_READ_FROM_SOCKET, 95) \ + M(CANNOT_WRITE_TO_SOCKET, 96) \ + M(CANNOT_READ_ALL_DATA_FROM_CHUNKED_INPUT, 97) \ + M(CANNOT_WRITE_TO_EMPTY_BLOCK_OUTPUT_STREAM, 98) \ + M(UNKNOWN_PACKET_FROM_CLIENT, 99) \ + M(UNKNOWN_PACKET_FROM_SERVER, 100) \ + M(UNEXPECTED_PACKET_FROM_CLIENT, 101) \ + M(UNEXPECTED_PACKET_FROM_SERVER, 102) \ + M(RECEIVED_DATA_FOR_WRONG_QUERY_ID, 103) \ + M(TOO_SMALL_BUFFER_SIZE, 104) \ + M(CANNOT_READ_HISTORY, 105) \ + M(CANNOT_APPEND_HISTORY, 106) \ + M(FILE_DOESNT_EXIST, 107) \ + M(NO_DATA_TO_INSERT, 108) \ + M(CANNOT_BLOCK_SIGNAL, 109) \ + M(CANNOT_UNBLOCK_SIGNAL, 110) \ + M(CANNOT_MANIPULATE_SIGSET, 111) \ + M(CANNOT_WAIT_FOR_SIGNAL, 112) \ + M(THERE_IS_NO_SESSION, 113) \ + M(CANNOT_CLOCK_GETTIME, 114) \ + M(UNKNOWN_SETTING, 115) \ + M(THERE_IS_NO_DEFAULT_VALUE, 116) \ + M(INCORRECT_DATA, 117) \ + M(ENGINE_REQUIRED, 119) \ + M(CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE, 120) \ + M(UNSUPPORTED_JOIN_KEYS, 121) \ + M(INCOMPATIBLE_COLUMNS, 122) \ + M(UNKNOWN_TYPE_OF_AST_NODE, 123) \ + M(INCORRECT_ELEMENT_OF_SET, 124) \ + M(INCORRECT_RESULT_OF_SCALAR_SUBQUERY, 125) \ + M(CANNOT_GET_RETURN_TYPE, 126) \ + M(ILLEGAL_INDEX, 127) \ + M(TOO_LARGE_ARRAY_SIZE, 128) \ + M(FUNCTION_IS_SPECIAL, 129) \ + M(CANNOT_READ_ARRAY_FROM_TEXT, 130) \ + M(TOO_LARGE_STRING_SIZE, 131) \ + M(AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS, 133) \ + M(PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS, 134) \ + M(ZERO_ARRAY_OR_TUPLE_INDEX, 135) \ + M(UNKNOWN_ELEMENT_IN_CONFIG, 137) \ + M(EXCESSIVE_ELEMENT_IN_CONFIG, 138) \ + M(NO_ELEMENTS_IN_CONFIG, 139) \ + M(ALL_REQUESTED_COLUMNS_ARE_MISSING, 140) \ + M(SAMPLING_NOT_SUPPORTED, 141) \ + M(NOT_FOUND_NODE, 142) \ + M(FOUND_MORE_THAN_ONE_NODE, 143) \ + M(FIRST_DATE_IS_BIGGER_THAN_LAST_DATE, 144) \ + M(UNKNOWN_OVERFLOW_MODE, 145) \ + M(QUERY_SECTION_DOESNT_MAKE_SENSE, 146) \ + M(NOT_FOUND_FUNCTION_ELEMENT_FOR_AGGREGATE, 147) \ + M(NOT_FOUND_RELATION_ELEMENT_FOR_CONDITION, 148) \ + M(NOT_FOUND_RHS_ELEMENT_FOR_CONDITION, 149) \ + M(EMPTY_LIST_OF_ATTRIBUTES_PASSED, 150) \ + M(INDEX_OF_COLUMN_IN_SORT_CLAUSE_IS_OUT_OF_RANGE, 151) \ + M(UNKNOWN_DIRECTION_OF_SORTING, 152) \ + M(ILLEGAL_DIVISION, 153) \ + M(AGGREGATE_FUNCTION_NOT_APPLICABLE, 154) \ + M(UNKNOWN_RELATION, 155) \ + M(DICTIONARIES_WAS_NOT_LOADED, 156) \ + M(ILLEGAL_OVERFLOW_MODE, 157) \ + M(TOO_MANY_ROWS, 158) \ + M(TIMEOUT_EXCEEDED, 159) \ + M(TOO_SLOW, 160) \ + M(TOO_MANY_COLUMNS, 161) \ + M(TOO_DEEP_SUBQUERIES, 162) \ + M(TOO_DEEP_PIPELINE, 163) \ + M(READONLY, 164) \ + M(TOO_MANY_TEMPORARY_COLUMNS, 165) \ + M(TOO_MANY_TEMPORARY_NON_CONST_COLUMNS, 166) \ + M(TOO_DEEP_AST, 167) \ + M(TOO_BIG_AST, 168) \ + M(BAD_TYPE_OF_FIELD, 169) \ + M(BAD_GET, 170) \ + M(CANNOT_CREATE_DIRECTORY, 172) \ + M(CANNOT_ALLOCATE_MEMORY, 173) \ + M(CYCLIC_ALIASES, 174) \ + M(CHUNK_NOT_FOUND, 176) \ + M(DUPLICATE_CHUNK_NAME, 177) \ + M(MULTIPLE_ALIASES_FOR_EXPRESSION, 178) \ + M(MULTIPLE_EXPRESSIONS_FOR_ALIAS, 179) \ + M(THERE_IS_NO_PROFILE, 180) \ + M(ILLEGAL_FINAL, 181) \ + M(ILLEGAL_PREWHERE, 182) \ + M(UNEXPECTED_EXPRESSION, 183) \ + M(ILLEGAL_AGGREGATION, 184) \ + M(UNSUPPORTED_MYISAM_BLOCK_TYPE, 185) \ + M(UNSUPPORTED_COLLATION_LOCALE, 186) \ + M(COLLATION_COMPARISON_FAILED, 187) \ + M(UNKNOWN_ACTION, 188) \ + M(TABLE_MUST_NOT_BE_CREATED_MANUALLY, 189) \ + M(SIZES_OF_ARRAYS_DOESNT_MATCH, 190) \ + M(SET_SIZE_LIMIT_EXCEEDED, 191) \ + M(UNKNOWN_USER, 192) \ + M(WRONG_PASSWORD, 193) \ + M(REQUIRED_PASSWORD, 194) \ + M(IP_ADDRESS_NOT_ALLOWED, 195) \ + M(UNKNOWN_ADDRESS_PATTERN_TYPE, 196) \ + M(SERVER_REVISION_IS_TOO_OLD, 197) \ + M(DNS_ERROR, 198) \ + M(UNKNOWN_QUOTA, 199) \ + M(QUOTA_DOESNT_ALLOW_KEYS, 200) \ + M(QUOTA_EXPIRED, 201) \ + M(TOO_MANY_SIMULTANEOUS_QUERIES, 202) \ + M(NO_FREE_CONNECTION, 203) \ + M(CANNOT_FSYNC, 204) \ + M(NESTED_TYPE_TOO_DEEP, 205) \ + M(ALIAS_REQUIRED, 206) \ + M(AMBIGUOUS_IDENTIFIER, 207) \ + M(EMPTY_NESTED_TABLE, 208) \ + M(SOCKET_TIMEOUT, 209) \ + M(NETWORK_ERROR, 210) \ + M(EMPTY_QUERY, 211) \ + M(UNKNOWN_LOAD_BALANCING, 212) \ + M(UNKNOWN_TOTALS_MODE, 213) \ + M(CANNOT_STATVFS, 214) \ + M(NOT_AN_AGGREGATE, 215) \ + M(QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING, 216) \ + M(CLIENT_HAS_CONNECTED_TO_WRONG_PORT, 217) \ + M(TABLE_IS_DROPPED, 218) \ + M(DATABASE_NOT_EMPTY, 219) \ + M(DUPLICATE_INTERSERVER_IO_ENDPOINT, 220) \ + M(NO_SUCH_INTERSERVER_IO_ENDPOINT, 221) \ + M(ADDING_REPLICA_TO_NON_EMPTY_TABLE, 222) \ + M(UNEXPECTED_AST_STRUCTURE, 223) \ + M(REPLICA_IS_ALREADY_ACTIVE, 224) \ + M(NO_ZOOKEEPER, 225) \ + M(NO_FILE_IN_DATA_PART, 226) \ + M(UNEXPECTED_FILE_IN_DATA_PART, 227) \ + M(BAD_SIZE_OF_FILE_IN_DATA_PART, 228) \ + M(QUERY_IS_TOO_LARGE, 229) \ + M(NOT_FOUND_EXPECTED_DATA_PART, 230) \ + M(TOO_MANY_UNEXPECTED_DATA_PARTS, 231) \ + M(NO_SUCH_DATA_PART, 232) \ + M(BAD_DATA_PART_NAME, 233) \ + M(NO_REPLICA_HAS_PART, 234) \ + M(DUPLICATE_DATA_PART, 235) \ + M(ABORTED, 236) \ + M(NO_REPLICA_NAME_GIVEN, 237) \ + M(FORMAT_VERSION_TOO_OLD, 238) \ + M(CANNOT_MUNMAP, 239) \ + M(CANNOT_MREMAP, 240) \ + M(MEMORY_LIMIT_EXCEEDED, 241) \ + M(TABLE_IS_READ_ONLY, 242) \ + M(NOT_ENOUGH_SPACE, 243) \ + M(UNEXPECTED_ZOOKEEPER_ERROR, 244) \ + M(CORRUPTED_DATA, 246) \ + M(INCORRECT_MARK, 247) \ + M(INVALID_PARTITION_VALUE, 248) \ + M(NOT_ENOUGH_BLOCK_NUMBERS, 250) \ + M(NO_SUCH_REPLICA, 251) \ + M(TOO_MANY_PARTS, 252) \ + M(REPLICA_IS_ALREADY_EXIST, 253) \ + M(NO_ACTIVE_REPLICAS, 254) \ + M(TOO_MANY_RETRIES_TO_FETCH_PARTS, 255) \ + M(PARTITION_ALREADY_EXISTS, 256) \ + M(PARTITION_DOESNT_EXIST, 257) \ + M(UNION_ALL_RESULT_STRUCTURES_MISMATCH, 258) \ + M(CLIENT_OUTPUT_FORMAT_SPECIFIED, 260) \ + M(UNKNOWN_BLOCK_INFO_FIELD, 261) \ + M(BAD_COLLATION, 262) \ + M(CANNOT_COMPILE_CODE, 263) \ + M(INCOMPATIBLE_TYPE_OF_JOIN, 264) \ + M(NO_AVAILABLE_REPLICA, 265) \ + M(MISMATCH_REPLICAS_DATA_SOURCES, 266) \ + M(STORAGE_DOESNT_SUPPORT_PARALLEL_REPLICAS, 267) \ + M(CPUID_ERROR, 268) \ + M(INFINITE_LOOP, 269) \ + M(CANNOT_COMPRESS, 270) \ + M(CANNOT_DECOMPRESS, 271) \ + M(CANNOT_IO_SUBMIT, 272) \ + M(CANNOT_IO_GETEVENTS, 273) \ + M(AIO_READ_ERROR, 274) \ + M(AIO_WRITE_ERROR, 275) \ + M(INDEX_NOT_USED, 277) \ + M(ALL_CONNECTION_TRIES_FAILED, 279) \ + M(NO_AVAILABLE_DATA, 280) \ + M(DICTIONARY_IS_EMPTY, 281) \ + M(INCORRECT_INDEX, 282) \ + M(UNKNOWN_DISTRIBUTED_PRODUCT_MODE, 283) \ + M(WRONG_GLOBAL_SUBQUERY, 284) \ + M(TOO_FEW_LIVE_REPLICAS, 285) \ + M(UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE, 286) \ + M(UNKNOWN_FORMAT_VERSION, 287) \ + M(DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED, 288) \ + M(REPLICA_IS_NOT_IN_QUORUM, 289) \ + M(LIMIT_EXCEEDED, 290) \ + M(DATABASE_ACCESS_DENIED, 291) \ + M(MONGODB_CANNOT_AUTHENTICATE, 293) \ + M(INVALID_BLOCK_EXTRA_INFO, 294) \ + M(RECEIVED_EMPTY_DATA, 295) \ + M(NO_REMOTE_SHARD_FOUND, 296) \ + M(SHARD_HAS_NO_CONNECTIONS, 297) \ + M(CANNOT_PIPE, 298) \ + M(CANNOT_FORK, 299) \ + M(CANNOT_DLSYM, 300) \ + M(CANNOT_CREATE_CHILD_PROCESS, 301) \ + M(CHILD_WAS_NOT_EXITED_NORMALLY, 302) \ + M(CANNOT_SELECT, 303) \ + M(CANNOT_WAITPID, 304) \ + M(TABLE_WAS_NOT_DROPPED, 305) \ + M(TOO_DEEP_RECURSION, 306) \ + M(TOO_MANY_BYTES, 307) \ + M(UNEXPECTED_NODE_IN_ZOOKEEPER, 308) \ + M(FUNCTION_CANNOT_HAVE_PARAMETERS, 309) \ + M(INVALID_SHARD_WEIGHT, 317) \ + M(INVALID_CONFIG_PARAMETER, 318) \ + M(UNKNOWN_STATUS_OF_INSERT, 319) \ + M(VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, 321) \ + M(BARRIER_TIMEOUT, 335) \ + M(UNKNOWN_DATABASE_ENGINE, 336) \ + M(DDL_GUARD_IS_ACTIVE, 337) \ + M(UNFINISHED, 341) \ + M(METADATA_MISMATCH, 342) \ + M(SUPPORT_IS_DISABLED, 344) \ + M(TABLE_DIFFERS_TOO_MUCH, 345) \ + M(CANNOT_CONVERT_CHARSET, 346) \ + M(CANNOT_LOAD_CONFIG, 347) \ + M(CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN, 349) \ + M(INCOMPATIBLE_SOURCE_TABLES, 350) \ + M(AMBIGUOUS_TABLE_NAME, 351) \ + M(AMBIGUOUS_COLUMN_NAME, 352) \ + M(INDEX_OF_POSITIONAL_ARGUMENT_IS_OUT_OF_RANGE, 353) \ + M(ZLIB_INFLATE_FAILED, 354) \ + M(ZLIB_DEFLATE_FAILED, 355) \ + M(BAD_LAMBDA, 356) \ + M(RESERVED_IDENTIFIER_NAME, 357) \ + M(INTO_OUTFILE_NOT_ALLOWED, 358) \ + M(TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT, 359) \ + M(CANNOT_CREATE_CHARSET_CONVERTER, 360) \ + M(SEEK_POSITION_OUT_OF_BOUND, 361) \ + M(CURRENT_WRITE_BUFFER_IS_EXHAUSTED, 362) \ + M(CANNOT_CREATE_IO_BUFFER, 363) \ + M(RECEIVED_ERROR_TOO_MANY_REQUESTS, 364) \ + M(SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT, 366) \ + M(TOO_MANY_FETCHES, 367) \ + M(ALL_REPLICAS_ARE_STALE, 369) \ + M(DATA_TYPE_CANNOT_BE_USED_IN_TABLES, 370) \ + M(INCONSISTENT_CLUSTER_DEFINITION, 371) \ + M(SESSION_NOT_FOUND, 372) \ + M(SESSION_IS_LOCKED, 373) \ + M(INVALID_SESSION_TIMEOUT, 374) \ + M(CANNOT_DLOPEN, 375) \ + M(CANNOT_PARSE_UUID, 376) \ + M(ILLEGAL_SYNTAX_FOR_DATA_TYPE, 377) \ + M(DATA_TYPE_CANNOT_HAVE_ARGUMENTS, 378) \ + M(UNKNOWN_STATUS_OF_DISTRIBUTED_DDL_TASK, 379) \ + M(CANNOT_KILL, 380) \ + M(HTTP_LENGTH_REQUIRED, 381) \ + M(CANNOT_LOAD_CATBOOST_MODEL, 382) \ + M(CANNOT_APPLY_CATBOOST_MODEL, 383) \ + M(PART_IS_TEMPORARILY_LOCKED, 384) \ + M(MULTIPLE_STREAMS_REQUIRED, 385) \ + M(NO_COMMON_TYPE, 386) \ + M(DICTIONARY_ALREADY_EXISTS, 387) \ + M(CANNOT_ASSIGN_OPTIMIZE, 388) \ + M(INSERT_WAS_DEDUPLICATED, 389) \ + M(CANNOT_GET_CREATE_TABLE_QUERY, 390) \ + M(EXTERNAL_LIBRARY_ERROR, 391) \ + M(QUERY_IS_PROHIBITED, 392) \ + M(THERE_IS_NO_QUERY, 393) \ + M(QUERY_WAS_CANCELLED, 394) \ + M(FUNCTION_THROW_IF_VALUE_IS_NON_ZERO, 395) \ + M(TOO_MANY_ROWS_OR_BYTES, 396) \ + M(QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, 397) \ + M(UNKNOWN_MUTATION_COMMAND, 398) \ + M(FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT, 399) \ + M(CANNOT_STAT, 400) \ + M(FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, 401) \ + M(CANNOT_IOSETUP, 402) \ + M(INVALID_JOIN_ON_EXPRESSION, 403) \ + M(BAD_ODBC_CONNECTION_STRING, 404) \ + M(PARTITION_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT, 405) \ + M(TOP_AND_LIMIT_TOGETHER, 406) \ + M(DECIMAL_OVERFLOW, 407) \ + M(BAD_REQUEST_PARAMETER, 408) \ + M(EXTERNAL_EXECUTABLE_NOT_FOUND, 409) \ + M(EXTERNAL_SERVER_IS_NOT_RESPONDING, 410) \ + M(PTHREAD_ERROR, 411) \ + M(NETLINK_ERROR, 412) \ + M(CANNOT_SET_SIGNAL_HANDLER, 413) \ + M(ALL_REPLICAS_LOST, 415) \ + M(REPLICA_STATUS_CHANGED, 416) \ + M(EXPECTED_ALL_OR_ANY, 417) \ + M(UNKNOWN_JOIN, 418) \ + M(MULTIPLE_ASSIGNMENTS_TO_COLUMN, 419) \ + M(CANNOT_UPDATE_COLUMN, 420) \ + M(CANNOT_ADD_DIFFERENT_AGGREGATE_STATES, 421) \ + M(UNSUPPORTED_URI_SCHEME, 422) \ + M(CANNOT_GETTIMEOFDAY, 423) \ + M(CANNOT_LINK, 424) \ + M(SYSTEM_ERROR, 425) \ + M(CANNOT_COMPILE_REGEXP, 427) \ + M(UNKNOWN_LOG_LEVEL, 428) \ + M(FAILED_TO_GETPWUID, 429) \ + M(MISMATCHING_USERS_FOR_PROCESS_AND_DATA, 430) \ + M(ILLEGAL_SYNTAX_FOR_CODEC_TYPE, 431) \ + M(UNKNOWN_CODEC, 432) \ + M(ILLEGAL_CODEC_PARAMETER, 433) \ + M(CANNOT_PARSE_PROTOBUF_SCHEMA, 434) \ + M(NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD, 435) \ + M(PROTOBUF_BAD_CAST, 436) \ + M(PROTOBUF_FIELD_NOT_REPEATED, 437) \ + M(DATA_TYPE_CANNOT_BE_PROMOTED, 438) \ + M(CANNOT_SCHEDULE_TASK, 439) \ + M(INVALID_LIMIT_EXPRESSION, 440) \ + M(CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING, 441) \ + M(BAD_DATABASE_FOR_TEMPORARY_TABLE, 442) \ + M(NO_COMMON_COLUMNS_WITH_PROTOBUF_SCHEMA, 443) \ + M(UNKNOWN_PROTOBUF_FORMAT, 444) \ + M(CANNOT_MPROTECT, 445) \ + M(FUNCTION_NOT_ALLOWED, 446) \ + M(HYPERSCAN_CANNOT_SCAN_TEXT, 447) \ + M(BROTLI_READ_FAILED, 448) \ + M(BROTLI_WRITE_FAILED, 449) \ + M(BAD_TTL_EXPRESSION, 450) \ + M(BAD_TTL_FILE, 451) \ + M(SETTING_CONSTRAINT_VIOLATION, 452) \ + M(MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES, 453) \ + M(OPENSSL_ERROR, 454) \ + M(SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY, 455) \ + M(UNKNOWN_QUERY_PARAMETER, 456) \ + M(BAD_QUERY_PARAMETER, 457) \ + M(CANNOT_UNLINK, 458) \ + M(CANNOT_SET_THREAD_PRIORITY, 459) \ + M(CANNOT_CREATE_TIMER, 460) \ + M(CANNOT_SET_TIMER_PERIOD, 461) \ + M(CANNOT_DELETE_TIMER, 462) \ + M(CANNOT_FCNTL, 463) \ + M(CANNOT_PARSE_ELF, 464) \ + M(CANNOT_PARSE_DWARF, 465) \ + M(INSECURE_PATH, 466) \ + M(CANNOT_PARSE_BOOL, 467) \ + M(CANNOT_PTHREAD_ATTR, 468) \ + M(VIOLATED_CONSTRAINT, 469) \ + M(QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW, 470) \ + M(INVALID_SETTING_VALUE, 471) \ + M(READONLY_SETTING, 472) \ + M(DEADLOCK_AVOIDED, 473) \ + M(INVALID_TEMPLATE_FORMAT, 474) \ + M(INVALID_WITH_FILL_EXPRESSION, 475) \ + M(WITH_TIES_WITHOUT_ORDER_BY, 476) \ + M(INVALID_USAGE_OF_INPUT, 477) \ + M(UNKNOWN_POLICY, 478) \ + M(UNKNOWN_DISK, 479) \ + M(UNKNOWN_PROTOCOL, 480) \ + M(PATH_ACCESS_DENIED, 481) \ + M(DICTIONARY_ACCESS_DENIED, 482) \ + M(TOO_MANY_REDIRECTS, 483) \ + M(INTERNAL_REDIS_ERROR, 484) \ + M(SCALAR_ALREADY_EXISTS, 485) \ + M(CANNOT_GET_CREATE_DICTIONARY_QUERY, 487) \ + M(UNKNOWN_DICTIONARY, 488) \ + M(INCORRECT_DICTIONARY_DEFINITION, 489) \ + M(CANNOT_FORMAT_DATETIME, 490) \ + M(UNACCEPTABLE_URL, 491) \ + M(ACCESS_ENTITY_NOT_FOUND, 492) \ + M(ACCESS_ENTITY_ALREADY_EXISTS, 493) \ + M(ACCESS_ENTITY_FOUND_DUPLICATES, 494) \ + M(ACCESS_STORAGE_READONLY, 495) \ + M(QUOTA_REQUIRES_CLIENT_KEY, 496) \ + M(ACCESS_DENIED, 497) \ + M(LIMIT_BY_WITH_TIES_IS_NOT_SUPPORTED, 498) \ + M(S3_ERROR, 499) \ + M(CANNOT_CREATE_DATABASE, 501) \ + M(CANNOT_SIGQUEUE, 502) \ + M(AGGREGATE_FUNCTION_THROW, 503) \ + M(FILE_ALREADY_EXISTS, 504) \ + M(CANNOT_DELETE_DIRECTORY, 505) \ + M(UNEXPECTED_ERROR_CODE, 506) \ + M(UNABLE_TO_SKIP_UNUSED_SHARDS, 507) \ + M(UNKNOWN_ACCESS_TYPE, 508) \ + M(INVALID_GRANT, 509) \ + M(CACHE_DICTIONARY_UPDATE_FAIL, 510) \ + M(UNKNOWN_ROLE, 511) \ + M(SET_NON_GRANTED_ROLE, 512) \ + M(UNKNOWN_PART_TYPE, 513) \ + M(ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND, 514) \ + M(INCORRECT_ACCESS_ENTITY_DEFINITION, 515) \ + M(AUTHENTICATION_FAILED, 516) \ + M(CANNOT_ASSIGN_ALTER, 517) \ + M(CANNOT_COMMIT_OFFSET, 518) \ + M(NO_REMOTE_SHARD_AVAILABLE, 519) \ + M(CANNOT_DETACH_DICTIONARY_AS_TABLE, 520) \ + M(ATOMIC_RENAME_FAIL, 521) \ + M(UNKNOWN_ROW_POLICY, 523) \ + M(ALTER_OF_COLUMN_IS_FORBIDDEN, 524) \ + M(INCORRECT_DISK_INDEX, 525) \ + M(UNKNOWN_VOLUME_TYPE, 526) \ + M(NO_SUITABLE_FUNCTION_IMPLEMENTATION, 527) \ + M(CASSANDRA_INTERNAL_ERROR, 528) \ + M(NOT_A_LEADER, 529) \ + M(CANNOT_CONNECT_RABBITMQ, 530) \ + M(CANNOT_FSTAT, 531) \ + M(LDAP_ERROR, 532) \ + M(INCONSISTENT_RESERVATIONS, 533) \ + M(NO_RESERVATIONS_PROVIDED, 534) \ + M(UNKNOWN_RAID_TYPE, 535) \ + M(CANNOT_RESTORE_FROM_FIELD_DUMP, 536) \ + M(ILLEGAL_MYSQL_VARIABLE, 537) \ + M(MYSQL_SYNTAX_ERROR, 538) \ + M(CANNOT_BIND_RABBITMQ_EXCHANGE, 539) \ + M(CANNOT_DECLARE_RABBITMQ_EXCHANGE, 540) \ + M(CANNOT_CREATE_RABBITMQ_QUEUE_BINDING, 541) \ + M(CANNOT_REMOVE_RABBITMQ_EXCHANGE, 542) \ + M(UNKNOWN_MYSQL_DATATYPES_SUPPORT_LEVEL, 543) \ + M(ROW_AND_ROWS_TOGETHER, 544) \ + M(FIRST_AND_NEXT_TOGETHER, 545) \ + M(NO_ROW_DELIMITER, 546) \ + M(INVALID_RAID_TYPE, 547) \ + M(UNKNOWN_VOLUME, 548) \ + \ + M(KEEPER_EXCEPTION, 999) \ + M(POCO_EXCEPTION, 1000) \ + M(STD_EXCEPTION, 1001) \ + M(UNKNOWN_EXCEPTION, 1002) \ + \ + M(CONDITIONAL_TREE_PARENT_NOT_FOUND, 2001) \ + M(ILLEGAL_PROJECTION_MANIPULATOR, 2002) \ + M(UNRECOGNIZED_ARGUMENTS, 2003) +/* See DB::ErrorCodes::END */ + namespace DB { namespace ErrorCodes { - /** Previously, these constants were located in one enum. - * But in this case there is a problem: when you add a new constant, you need to recompile - * all translation units that use at least one constant (almost the whole project). - * Therefore it is made so that definitions of constants are located here, in one file, - * and their declaration are in different files, at the place of use. - */ + #define M(NAME, VALUE) extern const Value NAME = VALUE; + APPLY_FOR_ERROR_CODES(M) + #undef M - extern const int UNSUPPORTED_METHOD = 1; - extern const int UNSUPPORTED_PARAMETER = 2; - extern const int UNEXPECTED_END_OF_FILE = 3; - extern const int EXPECTED_END_OF_FILE = 4; - extern const int CANNOT_PARSE_TEXT = 6; - extern const int INCORRECT_NUMBER_OF_COLUMNS = 7; - extern const int THERE_IS_NO_COLUMN = 8; - extern const int SIZES_OF_COLUMNS_DOESNT_MATCH = 9; - extern const int NOT_FOUND_COLUMN_IN_BLOCK = 10; - extern const int POSITION_OUT_OF_BOUND = 11; - extern const int PARAMETER_OUT_OF_BOUND = 12; - extern const int SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH = 13; - extern const int DUPLICATE_COLUMN = 15; - extern const int NO_SUCH_COLUMN_IN_TABLE = 16; - extern const int DELIMITER_IN_STRING_LITERAL_DOESNT_MATCH = 17; - extern const int CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN = 18; - extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH = 19; - extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH = 20; - extern const int CANNOT_READ_ALL_DATA_FROM_TAB_SEPARATED_INPUT = 21; - extern const int CANNOT_PARSE_ALL_VALUE_FROM_TAB_SEPARATED_INPUT = 22; - extern const int CANNOT_READ_FROM_ISTREAM = 23; - extern const int CANNOT_WRITE_TO_OSTREAM = 24; - extern const int CANNOT_PARSE_ESCAPE_SEQUENCE = 25; - extern const int CANNOT_PARSE_QUOTED_STRING = 26; - extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED = 27; - extern const int CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER = 28; - extern const int CANNOT_PRINT_INTEGER = 29; - extern const int CANNOT_READ_SIZE_OF_COMPRESSED_CHUNK = 30; - extern const int CANNOT_READ_COMPRESSED_CHUNK = 31; - extern const int ATTEMPT_TO_READ_AFTER_EOF = 32; - extern const int CANNOT_READ_ALL_DATA = 33; - extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION = 34; - extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION = 35; - extern const int BAD_ARGUMENTS = 36; - extern const int UNKNOWN_ELEMENT_IN_AST = 37; - extern const int CANNOT_PARSE_DATE = 38; - extern const int TOO_LARGE_SIZE_COMPRESSED = 39; - extern const int CHECKSUM_DOESNT_MATCH = 40; - extern const int CANNOT_PARSE_DATETIME = 41; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH = 42; - extern const int ILLEGAL_TYPE_OF_ARGUMENT = 43; - extern const int ILLEGAL_COLUMN = 44; - extern const int ILLEGAL_NUMBER_OF_RESULT_COLUMNS = 45; - extern const int UNKNOWN_FUNCTION = 46; - extern const int UNKNOWN_IDENTIFIER = 47; - extern const int NOT_IMPLEMENTED = 48; - extern const int LOGICAL_ERROR = 49; - extern const int UNKNOWN_TYPE = 50; - extern const int EMPTY_LIST_OF_COLUMNS_QUERIED = 51; - extern const int COLUMN_QUERIED_MORE_THAN_ONCE = 52; - extern const int TYPE_MISMATCH = 53; - extern const int STORAGE_DOESNT_ALLOW_PARAMETERS = 54; - extern const int STORAGE_REQUIRES_PARAMETER = 55; - extern const int UNKNOWN_STORAGE = 56; - extern const int TABLE_ALREADY_EXISTS = 57; - extern const int TABLE_METADATA_ALREADY_EXISTS = 58; - extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER = 59; - extern const int UNKNOWN_TABLE = 60; - extern const int ONLY_FILTER_COLUMN_IN_BLOCK = 61; - extern const int SYNTAX_ERROR = 62; - extern const int UNKNOWN_AGGREGATE_FUNCTION = 63; - extern const int CANNOT_READ_AGGREGATE_FUNCTION_FROM_TEXT = 64; - extern const int CANNOT_WRITE_AGGREGATE_FUNCTION_AS_TEXT = 65; - extern const int NOT_A_COLUMN = 66; - extern const int ILLEGAL_KEY_OF_AGGREGATION = 67; - extern const int CANNOT_GET_SIZE_OF_FIELD = 68; - extern const int ARGUMENT_OUT_OF_BOUND = 69; - extern const int CANNOT_CONVERT_TYPE = 70; - extern const int CANNOT_WRITE_AFTER_END_OF_BUFFER = 71; - extern const int CANNOT_PARSE_NUMBER = 72; - extern const int UNKNOWN_FORMAT = 73; - extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR = 74; - extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR = 75; - extern const int CANNOT_OPEN_FILE = 76; - extern const int CANNOT_CLOSE_FILE = 77; - extern const int UNKNOWN_TYPE_OF_QUERY = 78; - extern const int INCORRECT_FILE_NAME = 79; - extern const int INCORRECT_QUERY = 80; - extern const int UNKNOWN_DATABASE = 81; - extern const int DATABASE_ALREADY_EXISTS = 82; - extern const int DIRECTORY_DOESNT_EXIST = 83; - extern const int DIRECTORY_ALREADY_EXISTS = 84; - extern const int FORMAT_IS_NOT_SUITABLE_FOR_INPUT = 85; - extern const int RECEIVED_ERROR_FROM_REMOTE_IO_SERVER = 86; - extern const int CANNOT_SEEK_THROUGH_FILE = 87; - extern const int CANNOT_TRUNCATE_FILE = 88; - extern const int UNKNOWN_COMPRESSION_METHOD = 89; - extern const int EMPTY_LIST_OF_COLUMNS_PASSED = 90; - extern const int SIZES_OF_MARKS_FILES_ARE_INCONSISTENT = 91; - extern const int EMPTY_DATA_PASSED = 92; - extern const int UNKNOWN_AGGREGATED_DATA_VARIANT = 93; - extern const int CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS = 94; - extern const int CANNOT_READ_FROM_SOCKET = 95; - extern const int CANNOT_WRITE_TO_SOCKET = 96; - extern const int CANNOT_READ_ALL_DATA_FROM_CHUNKED_INPUT = 97; - extern const int CANNOT_WRITE_TO_EMPTY_BLOCK_OUTPUT_STREAM = 98; - extern const int UNKNOWN_PACKET_FROM_CLIENT = 99; - extern const int UNKNOWN_PACKET_FROM_SERVER = 100; - extern const int UNEXPECTED_PACKET_FROM_CLIENT = 101; - extern const int UNEXPECTED_PACKET_FROM_SERVER = 102; - extern const int RECEIVED_DATA_FOR_WRONG_QUERY_ID = 103; - extern const int TOO_SMALL_BUFFER_SIZE = 104; - extern const int CANNOT_READ_HISTORY = 105; - extern const int CANNOT_APPEND_HISTORY = 106; - extern const int FILE_DOESNT_EXIST = 107; - extern const int NO_DATA_TO_INSERT = 108; - extern const int CANNOT_BLOCK_SIGNAL = 109; - extern const int CANNOT_UNBLOCK_SIGNAL = 110; - extern const int CANNOT_MANIPULATE_SIGSET = 111; - extern const int CANNOT_WAIT_FOR_SIGNAL = 112; - extern const int THERE_IS_NO_SESSION = 113; - extern const int CANNOT_CLOCK_GETTIME = 114; - extern const int UNKNOWN_SETTING = 115; - extern const int THERE_IS_NO_DEFAULT_VALUE = 116; - extern const int INCORRECT_DATA = 117; - extern const int ENGINE_REQUIRED = 119; - extern const int CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE = 120; - extern const int UNSUPPORTED_JOIN_KEYS = 121; - extern const int INCOMPATIBLE_COLUMNS = 122; - extern const int UNKNOWN_TYPE_OF_AST_NODE = 123; - extern const int INCORRECT_ELEMENT_OF_SET = 124; - extern const int INCORRECT_RESULT_OF_SCALAR_SUBQUERY = 125; - extern const int CANNOT_GET_RETURN_TYPE = 126; - extern const int ILLEGAL_INDEX = 127; - extern const int TOO_LARGE_ARRAY_SIZE = 128; - extern const int FUNCTION_IS_SPECIAL = 129; - extern const int CANNOT_READ_ARRAY_FROM_TEXT = 130; - extern const int TOO_LARGE_STRING_SIZE = 131; - extern const int AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS = 133; - extern const int PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS = 134; - extern const int ZERO_ARRAY_OR_TUPLE_INDEX = 135; - extern const int UNKNOWN_ELEMENT_IN_CONFIG = 137; - extern const int EXCESSIVE_ELEMENT_IN_CONFIG = 138; - extern const int NO_ELEMENTS_IN_CONFIG = 139; - extern const int ALL_REQUESTED_COLUMNS_ARE_MISSING = 140; - extern const int SAMPLING_NOT_SUPPORTED = 141; - extern const int NOT_FOUND_NODE = 142; - extern const int FOUND_MORE_THAN_ONE_NODE = 143; - extern const int FIRST_DATE_IS_BIGGER_THAN_LAST_DATE = 144; - extern const int UNKNOWN_OVERFLOW_MODE = 145; - extern const int QUERY_SECTION_DOESNT_MAKE_SENSE = 146; - extern const int NOT_FOUND_FUNCTION_ELEMENT_FOR_AGGREGATE = 147; - extern const int NOT_FOUND_RELATION_ELEMENT_FOR_CONDITION = 148; - extern const int NOT_FOUND_RHS_ELEMENT_FOR_CONDITION = 149; - extern const int EMPTY_LIST_OF_ATTRIBUTES_PASSED = 150; - extern const int INDEX_OF_COLUMN_IN_SORT_CLAUSE_IS_OUT_OF_RANGE = 151; - extern const int UNKNOWN_DIRECTION_OF_SORTING = 152; - extern const int ILLEGAL_DIVISION = 153; - extern const int AGGREGATE_FUNCTION_NOT_APPLICABLE = 154; - extern const int UNKNOWN_RELATION = 155; - extern const int DICTIONARIES_WAS_NOT_LOADED = 156; - extern const int ILLEGAL_OVERFLOW_MODE = 157; - extern const int TOO_MANY_ROWS = 158; - extern const int TIMEOUT_EXCEEDED = 159; - extern const int TOO_SLOW = 160; - extern const int TOO_MANY_COLUMNS = 161; - extern const int TOO_DEEP_SUBQUERIES = 162; - extern const int TOO_DEEP_PIPELINE = 163; - extern const int READONLY = 164; - extern const int TOO_MANY_TEMPORARY_COLUMNS = 165; - extern const int TOO_MANY_TEMPORARY_NON_CONST_COLUMNS = 166; - extern const int TOO_DEEP_AST = 167; - extern const int TOO_BIG_AST = 168; - extern const int BAD_TYPE_OF_FIELD = 169; - extern const int BAD_GET = 170; - extern const int CANNOT_CREATE_DIRECTORY = 172; - extern const int CANNOT_ALLOCATE_MEMORY = 173; - extern const int CYCLIC_ALIASES = 174; - extern const int CHUNK_NOT_FOUND = 176; - extern const int DUPLICATE_CHUNK_NAME = 177; - extern const int MULTIPLE_ALIASES_FOR_EXPRESSION = 178; - extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS = 179; - extern const int THERE_IS_NO_PROFILE = 180; - extern const int ILLEGAL_FINAL = 181; - extern const int ILLEGAL_PREWHERE = 182; - extern const int UNEXPECTED_EXPRESSION = 183; - extern const int ILLEGAL_AGGREGATION = 184; - extern const int UNSUPPORTED_MYISAM_BLOCK_TYPE = 185; - extern const int UNSUPPORTED_COLLATION_LOCALE = 186; - extern const int COLLATION_COMPARISON_FAILED = 187; - extern const int UNKNOWN_ACTION = 188; - extern const int TABLE_MUST_NOT_BE_CREATED_MANUALLY = 189; - extern const int SIZES_OF_ARRAYS_DOESNT_MATCH = 190; - extern const int SET_SIZE_LIMIT_EXCEEDED = 191; - extern const int UNKNOWN_USER = 192; - extern const int WRONG_PASSWORD = 193; - extern const int REQUIRED_PASSWORD = 194; - extern const int IP_ADDRESS_NOT_ALLOWED = 195; - extern const int UNKNOWN_ADDRESS_PATTERN_TYPE = 196; - extern const int SERVER_REVISION_IS_TOO_OLD = 197; - extern const int DNS_ERROR = 198; - extern const int UNKNOWN_QUOTA = 199; - extern const int QUOTA_DOESNT_ALLOW_KEYS = 200; - extern const int QUOTA_EXPIRED = 201; - extern const int TOO_MANY_SIMULTANEOUS_QUERIES = 202; - extern const int NO_FREE_CONNECTION = 203; - extern const int CANNOT_FSYNC = 204; - extern const int NESTED_TYPE_TOO_DEEP = 205; - extern const int ALIAS_REQUIRED = 206; - extern const int AMBIGUOUS_IDENTIFIER = 207; - extern const int EMPTY_NESTED_TABLE = 208; - extern const int SOCKET_TIMEOUT = 209; - extern const int NETWORK_ERROR = 210; - extern const int EMPTY_QUERY = 211; - extern const int UNKNOWN_LOAD_BALANCING = 212; - extern const int UNKNOWN_TOTALS_MODE = 213; - extern const int CANNOT_STATVFS = 214; - extern const int NOT_AN_AGGREGATE = 215; - extern const int QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING = 216; - extern const int CLIENT_HAS_CONNECTED_TO_WRONG_PORT = 217; - extern const int TABLE_IS_DROPPED = 218; - extern const int DATABASE_NOT_EMPTY = 219; - extern const int DUPLICATE_INTERSERVER_IO_ENDPOINT = 220; - extern const int NO_SUCH_INTERSERVER_IO_ENDPOINT = 221; - extern const int ADDING_REPLICA_TO_NON_EMPTY_TABLE = 222; - extern const int UNEXPECTED_AST_STRUCTURE = 223; - extern const int REPLICA_IS_ALREADY_ACTIVE = 224; - extern const int NO_ZOOKEEPER = 225; - extern const int NO_FILE_IN_DATA_PART = 226; - extern const int UNEXPECTED_FILE_IN_DATA_PART = 227; - extern const int BAD_SIZE_OF_FILE_IN_DATA_PART = 228; - extern const int QUERY_IS_TOO_LARGE = 229; - extern const int NOT_FOUND_EXPECTED_DATA_PART = 230; - extern const int TOO_MANY_UNEXPECTED_DATA_PARTS = 231; - extern const int NO_SUCH_DATA_PART = 232; - extern const int BAD_DATA_PART_NAME = 233; - extern const int NO_REPLICA_HAS_PART = 234; - extern const int DUPLICATE_DATA_PART = 235; - extern const int ABORTED = 236; - extern const int NO_REPLICA_NAME_GIVEN = 237; - extern const int FORMAT_VERSION_TOO_OLD = 238; - extern const int CANNOT_MUNMAP = 239; - extern const int CANNOT_MREMAP = 240; - extern const int MEMORY_LIMIT_EXCEEDED = 241; - extern const int TABLE_IS_READ_ONLY = 242; - extern const int NOT_ENOUGH_SPACE = 243; - extern const int UNEXPECTED_ZOOKEEPER_ERROR = 244; - extern const int CORRUPTED_DATA = 246; - extern const int INCORRECT_MARK = 247; - extern const int INVALID_PARTITION_VALUE = 248; - extern const int NOT_ENOUGH_BLOCK_NUMBERS = 250; - extern const int NO_SUCH_REPLICA = 251; - extern const int TOO_MANY_PARTS = 252; - extern const int REPLICA_IS_ALREADY_EXIST = 253; - extern const int NO_ACTIVE_REPLICAS = 254; - extern const int TOO_MANY_RETRIES_TO_FETCH_PARTS = 255; - extern const int PARTITION_ALREADY_EXISTS = 256; - extern const int PARTITION_DOESNT_EXIST = 257; - extern const int UNION_ALL_RESULT_STRUCTURES_MISMATCH = 258; - extern const int CLIENT_OUTPUT_FORMAT_SPECIFIED = 260; - extern const int UNKNOWN_BLOCK_INFO_FIELD = 261; - extern const int BAD_COLLATION = 262; - extern const int CANNOT_COMPILE_CODE = 263; - extern const int INCOMPATIBLE_TYPE_OF_JOIN = 264; - extern const int NO_AVAILABLE_REPLICA = 265; - extern const int MISMATCH_REPLICAS_DATA_SOURCES = 266; - extern const int STORAGE_DOESNT_SUPPORT_PARALLEL_REPLICAS = 267; - extern const int CPUID_ERROR = 268; - extern const int INFINITE_LOOP = 269; - extern const int CANNOT_COMPRESS = 270; - extern const int CANNOT_DECOMPRESS = 271; - extern const int CANNOT_IO_SUBMIT = 272; - extern const int CANNOT_IO_GETEVENTS = 273; - extern const int AIO_READ_ERROR = 274; - extern const int AIO_WRITE_ERROR = 275; - extern const int INDEX_NOT_USED = 277; - extern const int ALL_CONNECTION_TRIES_FAILED = 279; - extern const int NO_AVAILABLE_DATA = 280; - extern const int DICTIONARY_IS_EMPTY = 281; - extern const int INCORRECT_INDEX = 282; - extern const int UNKNOWN_DISTRIBUTED_PRODUCT_MODE = 283; - extern const int WRONG_GLOBAL_SUBQUERY = 284; - extern const int TOO_FEW_LIVE_REPLICAS = 285; - extern const int UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE = 286; - extern const int UNKNOWN_FORMAT_VERSION = 287; - extern const int DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED = 288; - extern const int REPLICA_IS_NOT_IN_QUORUM = 289; - extern const int LIMIT_EXCEEDED = 290; - extern const int DATABASE_ACCESS_DENIED = 291; - extern const int MONGODB_CANNOT_AUTHENTICATE = 293; - extern const int INVALID_BLOCK_EXTRA_INFO = 294; - extern const int RECEIVED_EMPTY_DATA = 295; - extern const int NO_REMOTE_SHARD_FOUND = 296; - extern const int SHARD_HAS_NO_CONNECTIONS = 297; - extern const int CANNOT_PIPE = 298; - extern const int CANNOT_FORK = 299; - extern const int CANNOT_DLSYM = 300; - extern const int CANNOT_CREATE_CHILD_PROCESS = 301; - extern const int CHILD_WAS_NOT_EXITED_NORMALLY = 302; - extern const int CANNOT_SELECT = 303; - extern const int CANNOT_WAITPID = 304; - extern const int TABLE_WAS_NOT_DROPPED = 305; - extern const int TOO_DEEP_RECURSION = 306; - extern const int TOO_MANY_BYTES = 307; - extern const int UNEXPECTED_NODE_IN_ZOOKEEPER = 308; - extern const int FUNCTION_CANNOT_HAVE_PARAMETERS = 309; - extern const int INVALID_SHARD_WEIGHT = 317; - extern const int INVALID_CONFIG_PARAMETER = 318; - extern const int UNKNOWN_STATUS_OF_INSERT = 319; - extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE = 321; - extern const int BARRIER_TIMEOUT = 335; - extern const int UNKNOWN_DATABASE_ENGINE = 336; - extern const int DDL_GUARD_IS_ACTIVE = 337; - extern const int UNFINISHED = 341; - extern const int METADATA_MISMATCH = 342; - extern const int SUPPORT_IS_DISABLED = 344; - extern const int TABLE_DIFFERS_TOO_MUCH = 345; - extern const int CANNOT_CONVERT_CHARSET = 346; - extern const int CANNOT_LOAD_CONFIG = 347; - extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN = 349; - extern const int INCOMPATIBLE_SOURCE_TABLES = 350; - extern const int AMBIGUOUS_TABLE_NAME = 351; - extern const int AMBIGUOUS_COLUMN_NAME = 352; - extern const int INDEX_OF_POSITIONAL_ARGUMENT_IS_OUT_OF_RANGE = 353; - extern const int ZLIB_INFLATE_FAILED = 354; - extern const int ZLIB_DEFLATE_FAILED = 355; - extern const int BAD_LAMBDA = 356; - extern const int RESERVED_IDENTIFIER_NAME = 357; - extern const int INTO_OUTFILE_NOT_ALLOWED = 358; - extern const int TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT = 359; - extern const int CANNOT_CREATE_CHARSET_CONVERTER = 360; - extern const int SEEK_POSITION_OUT_OF_BOUND = 361; - extern const int CURRENT_WRITE_BUFFER_IS_EXHAUSTED = 362; - extern const int CANNOT_CREATE_IO_BUFFER = 363; - extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS = 364; - extern const int SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT = 366; - extern const int TOO_MANY_FETCHES = 367; - extern const int ALL_REPLICAS_ARE_STALE = 369; - extern const int DATA_TYPE_CANNOT_BE_USED_IN_TABLES = 370; - extern const int INCONSISTENT_CLUSTER_DEFINITION = 371; - extern const int SESSION_NOT_FOUND = 372; - extern const int SESSION_IS_LOCKED = 373; - extern const int INVALID_SESSION_TIMEOUT = 374; - extern const int CANNOT_DLOPEN = 375; - extern const int CANNOT_PARSE_UUID = 376; - extern const int ILLEGAL_SYNTAX_FOR_DATA_TYPE = 377; - extern const int DATA_TYPE_CANNOT_HAVE_ARGUMENTS = 378; - extern const int UNKNOWN_STATUS_OF_DISTRIBUTED_DDL_TASK = 379; - extern const int CANNOT_KILL = 380; - extern const int HTTP_LENGTH_REQUIRED = 381; - extern const int CANNOT_LOAD_CATBOOST_MODEL = 382; - extern const int CANNOT_APPLY_CATBOOST_MODEL = 383; - extern const int PART_IS_TEMPORARILY_LOCKED = 384; - extern const int MULTIPLE_STREAMS_REQUIRED = 385; - extern const int NO_COMMON_TYPE = 386; - extern const int DICTIONARY_ALREADY_EXISTS = 387; - extern const int CANNOT_ASSIGN_OPTIMIZE = 388; - extern const int INSERT_WAS_DEDUPLICATED = 389; - extern const int CANNOT_GET_CREATE_TABLE_QUERY = 390; - extern const int EXTERNAL_LIBRARY_ERROR = 391; - extern const int QUERY_IS_PROHIBITED = 392; - extern const int THERE_IS_NO_QUERY = 393; - extern const int QUERY_WAS_CANCELLED = 394; - extern const int FUNCTION_THROW_IF_VALUE_IS_NON_ZERO = 395; - extern const int TOO_MANY_ROWS_OR_BYTES = 396; - extern const int QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW = 397; - extern const int UNKNOWN_MUTATION_COMMAND = 398; - extern const int FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT = 399; - extern const int CANNOT_STAT = 400; - extern const int FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME = 401; - extern const int CANNOT_IOSETUP = 402; - extern const int INVALID_JOIN_ON_EXPRESSION = 403; - extern const int BAD_ODBC_CONNECTION_STRING = 404; - extern const int PARTITION_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT = 405; - extern const int TOP_AND_LIMIT_TOGETHER = 406; - extern const int DECIMAL_OVERFLOW = 407; - extern const int BAD_REQUEST_PARAMETER = 408; - extern const int EXTERNAL_EXECUTABLE_NOT_FOUND = 409; - extern const int EXTERNAL_SERVER_IS_NOT_RESPONDING = 410; - extern const int PTHREAD_ERROR = 411; - extern const int NETLINK_ERROR = 412; - extern const int CANNOT_SET_SIGNAL_HANDLER = 413; - extern const int ALL_REPLICAS_LOST = 415; - extern const int REPLICA_STATUS_CHANGED = 416; - extern const int EXPECTED_ALL_OR_ANY = 417; - extern const int UNKNOWN_JOIN = 418; - extern const int MULTIPLE_ASSIGNMENTS_TO_COLUMN = 419; - extern const int CANNOT_UPDATE_COLUMN = 420; - extern const int CANNOT_ADD_DIFFERENT_AGGREGATE_STATES = 421; - extern const int UNSUPPORTED_URI_SCHEME = 422; - extern const int CANNOT_GETTIMEOFDAY = 423; - extern const int CANNOT_LINK = 424; - extern const int SYSTEM_ERROR = 425; - extern const int CANNOT_COMPILE_REGEXP = 427; - extern const int UNKNOWN_LOG_LEVEL = 428; - extern const int FAILED_TO_GETPWUID = 429; - extern const int MISMATCHING_USERS_FOR_PROCESS_AND_DATA = 430; - extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE = 431; - extern const int UNKNOWN_CODEC = 432; - extern const int ILLEGAL_CODEC_PARAMETER = 433; - extern const int CANNOT_PARSE_PROTOBUF_SCHEMA = 434; - extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD = 435; - extern const int PROTOBUF_BAD_CAST = 436; - extern const int PROTOBUF_FIELD_NOT_REPEATED = 437; - extern const int DATA_TYPE_CANNOT_BE_PROMOTED = 438; - extern const int CANNOT_SCHEDULE_TASK = 439; - extern const int INVALID_LIMIT_EXPRESSION = 440; - extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING = 441; - extern const int BAD_DATABASE_FOR_TEMPORARY_TABLE = 442; - extern const int NO_COMMON_COLUMNS_WITH_PROTOBUF_SCHEMA = 443; - extern const int UNKNOWN_PROTOBUF_FORMAT = 444; - extern const int CANNOT_MPROTECT = 445; - extern const int FUNCTION_NOT_ALLOWED = 446; - extern const int HYPERSCAN_CANNOT_SCAN_TEXT = 447; - extern const int BROTLI_READ_FAILED = 448; - extern const int BROTLI_WRITE_FAILED = 449; - extern const int BAD_TTL_EXPRESSION = 450; - extern const int BAD_TTL_FILE = 451; - extern const int SETTING_CONSTRAINT_VIOLATION = 452; - extern const int MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES = 453; - extern const int OPENSSL_ERROR = 454; - extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY = 455; - extern const int UNKNOWN_QUERY_PARAMETER = 456; - extern const int BAD_QUERY_PARAMETER = 457; - extern const int CANNOT_UNLINK = 458; - extern const int CANNOT_SET_THREAD_PRIORITY = 459; - extern const int CANNOT_CREATE_TIMER = 460; - extern const int CANNOT_SET_TIMER_PERIOD = 461; - extern const int CANNOT_DELETE_TIMER = 462; - extern const int CANNOT_FCNTL = 463; - extern const int CANNOT_PARSE_ELF = 464; - extern const int CANNOT_PARSE_DWARF = 465; - extern const int INSECURE_PATH = 466; - extern const int CANNOT_PARSE_BOOL = 467; - extern const int CANNOT_PTHREAD_ATTR = 468; - extern const int VIOLATED_CONSTRAINT = 469; - extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW = 470; - extern const int INVALID_SETTING_VALUE = 471; - extern const int READONLY_SETTING = 472; - extern const int DEADLOCK_AVOIDED = 473; - extern const int INVALID_TEMPLATE_FORMAT = 474; - extern const int INVALID_WITH_FILL_EXPRESSION = 475; - extern const int WITH_TIES_WITHOUT_ORDER_BY = 476; - extern const int INVALID_USAGE_OF_INPUT = 477; - extern const int UNKNOWN_POLICY = 478; - extern const int UNKNOWN_DISK = 479; - extern const int UNKNOWN_PROTOCOL = 480; - extern const int PATH_ACCESS_DENIED = 481; - extern const int DICTIONARY_ACCESS_DENIED = 482; - extern const int TOO_MANY_REDIRECTS = 483; - extern const int INTERNAL_REDIS_ERROR = 484; - extern const int SCALAR_ALREADY_EXISTS = 485; - extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY = 487; - extern const int UNKNOWN_DICTIONARY = 488; - extern const int INCORRECT_DICTIONARY_DEFINITION = 489; - extern const int CANNOT_FORMAT_DATETIME = 490; - extern const int UNACCEPTABLE_URL = 491; - extern const int ACCESS_ENTITY_NOT_FOUND = 492; - extern const int ACCESS_ENTITY_ALREADY_EXISTS = 493; - extern const int ACCESS_ENTITY_FOUND_DUPLICATES = 494; - extern const int ACCESS_STORAGE_READONLY = 495; - extern const int QUOTA_REQUIRES_CLIENT_KEY = 496; - extern const int ACCESS_DENIED = 497; - extern const int LIMIT_BY_WITH_TIES_IS_NOT_SUPPORTED = 498; - extern const int S3_ERROR = 499; - extern const int CANNOT_CREATE_DATABASE = 501; - extern const int CANNOT_SIGQUEUE = 502; - extern const int AGGREGATE_FUNCTION_THROW = 503; - extern const int FILE_ALREADY_EXISTS = 504; - extern const int CANNOT_DELETE_DIRECTORY = 505; - extern const int UNEXPECTED_ERROR_CODE = 506; - extern const int UNABLE_TO_SKIP_UNUSED_SHARDS = 507; - extern const int UNKNOWN_ACCESS_TYPE = 508; - extern const int INVALID_GRANT = 509; - extern const int CACHE_DICTIONARY_UPDATE_FAIL = 510; - extern const int UNKNOWN_ROLE = 511; - extern const int SET_NON_GRANTED_ROLE = 512; - extern const int UNKNOWN_PART_TYPE = 513; - extern const int ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND = 514; - extern const int INCORRECT_ACCESS_ENTITY_DEFINITION = 515; - extern const int AUTHENTICATION_FAILED = 516; - extern const int CANNOT_ASSIGN_ALTER = 517; - extern const int CANNOT_COMMIT_OFFSET = 518; - extern const int NO_REMOTE_SHARD_AVAILABLE = 519; - extern const int CANNOT_DETACH_DICTIONARY_AS_TABLE = 520; - extern const int ATOMIC_RENAME_FAIL = 521; - extern const int UNKNOWN_ROW_POLICY = 523; - extern const int ALTER_OF_COLUMN_IS_FORBIDDEN = 524; - extern const int INCORRECT_DISK_INDEX = 525; - extern const int UNKNOWN_VOLUME_TYPE = 526; - extern const int NO_SUITABLE_FUNCTION_IMPLEMENTATION = 527; - extern const int CASSANDRA_INTERNAL_ERROR = 528; - extern const int NOT_A_LEADER = 529; - extern const int CANNOT_CONNECT_RABBITMQ = 530; - extern const int CANNOT_FSTAT = 531; - extern const int LDAP_ERROR = 532; - extern const int INCONSISTENT_RESERVATIONS = 533; - extern const int NO_RESERVATIONS_PROVIDED = 534; - extern const int UNKNOWN_RAID_TYPE = 535; - extern const int CANNOT_RESTORE_FROM_FIELD_DUMP = 536; - extern const int ILLEGAL_MYSQL_VARIABLE = 537; - extern const int MYSQL_SYNTAX_ERROR = 538; - extern const int CANNOT_BIND_RABBITMQ_EXCHANGE = 539; - extern const int CANNOT_DECLARE_RABBITMQ_EXCHANGE = 540; - extern const int CANNOT_CREATE_RABBITMQ_QUEUE_BINDING = 541; - extern const int CANNOT_REMOVE_RABBITMQ_EXCHANGE = 542; - extern const int UNKNOWN_MYSQL_DATATYPES_SUPPORT_LEVEL = 543; - extern const int ROW_AND_ROWS_TOGETHER = 544; - extern const int FIRST_AND_NEXT_TOGETHER = 545; - extern const int NO_ROW_DELIMITER = 546; - extern const int INVALID_RAID_TYPE = 547; - extern const int UNKNOWN_VOLUME = 548; + constexpr Value END = 3000; + std::atomic values[END+1] {}; - extern const int KEEPER_EXCEPTION = 999; - extern const int POCO_EXCEPTION = 1000; - extern const int STD_EXCEPTION = 1001; - extern const int UNKNOWN_EXCEPTION = 1002; + struct ErrorCodesNames + { + const char * strings[END+1]; + ErrorCodesNames() + { + #define M(NAME, VALUE) strings[VALUE] = #NAME; + APPLY_FOR_ERROR_CODES(M) + #undef M + } + } error_codes_names; - extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND = 2001; - extern const int ILLEGAL_PROJECTION_MANIPULATOR = 2002; - extern const int UNRECOGNIZED_ARGUMENTS = 2003; + std::string_view getName(ErrorCode error_code) + { + if (error_code >= END) + return std::string_view(); + const char * name = error_codes_names.strings[error_code]; + if (!name) + return std::string_view(); + return std::string_view(name); + } + + ErrorCode end() { return END+1; } } } diff --git a/src/Common/ErrorCodes.h b/src/Common/ErrorCodes.h new file mode 100644 index 00000000000..008cf018bd6 --- /dev/null +++ b/src/Common/ErrorCodes.h @@ -0,0 +1,41 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +/** Allows to count number of simultaneously happening error codes. + * See also Exception.cpp for incrementing part. + */ + +namespace DB +{ + +namespace ErrorCodes +{ + /// ErrorCode identifier (index in array). + using ErrorCode = size_t; + using Value = int; + + /// Get name of error_code by identifier. + /// Returns statically allocated string. + std::string_view getName(ErrorCode event); + + /// ErrorCode identifier -> current value of error_code. + extern std::atomic values[]; + + /// Get index just after last error_code identifier. + ErrorCode end(); + + /// Add value for specified error_code. + inline void increment(ErrorCode error_code) + { + error_code = std::min(error_code, end()-1); + values[error_code].fetch_add(1, std::memory_order_relaxed); + } +} + +} diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 4b9bb595ce6..c57726faa7c 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -12,19 +12,15 @@ #include #include #include -#include #include #include -#include +#include #include -#include #if !defined(ARCADIA_BUILD) # include #endif -extern HashMap, DefaultHash> error_codes_count; - namespace DB { @@ -50,7 +46,7 @@ Exception::Exception(const std::string & msg, int code) LOG_FATAL(&Poco::Logger::root(), "Logical error: '{}'.", msg); abort(); } - ++error_codes_count[code]; + ErrorCodes::increment(code); #endif } diff --git a/src/Common/errorCodes.sh b/src/Common/errorCodes.sh deleted file mode 100755 index 3a4520e2dad..00000000000 --- a/src/Common/errorCodes.sh +++ /dev/null @@ -1,93 +0,0 @@ -#!/usr/bin/env bash - -# Parse src/Common/ErrorCodes.cpp -# And generate src/Functions/errorCodes.generated.cpp -# For errorCode() function. -# -# Later it may contain some description of the error. - -set -e -set -o pipefail - -CUR_DIR="$(readlink -f "$(dirname "${BASH_SOURCE[0]}")")" -ERROR_CODES_IN_FILE=${ERROR_CODES_IN_FILE=$CUR_DIR/../Common/ErrorCodes.cpp} -ERROR_CODES_OUT_FILE=${ERROR_CODES_OUT_FILE=$CUR_DIR/errorCodes.generated.cpp} -CXX=${CXX=g++} - -trap 'rm -f $TMP_FILE' EXIT -TMP_FILE="$(mktemp clichouse_generate_errorCodes_XXXXXXXX.cpp)" - -function parse_ErrorCodes() -{ - # This is the simplest command that can be written to parse the file - # And it does not requires any extra tools and works everywhere where you have g++/clang++ - # - # Generate: - # - # CODE VAR_NAME - # - $CXX -E "$ERROR_CODES_IN_FILE" | { - awk -F '[ =;]*' '/extern const int / { print $(NF-1), $(NF-2); }' - } -} -function parse_for_errorCodeToName() -{ - parse_ErrorCodes | awk '{ printf(" case %s: return std::string_view(\"%s\");\n", $1, $2); }' -} -function generate_errorCodeToName() -{ - cat < - -std::string_view errorCodeToName(int code) -{ - switch (code) - { - case 0: return std::string_view("OK"); -$(parse_for_errorCodeToName) - default: return std::string_view(""); - } -}; - -EOL -} - -function parse_for_error_codes_count() -{ - parse_ErrorCodes | awk '{ printf(" error_codes_count[%s] = 0; /* %s */\n", $1, $2); }' -} -function generate_error_codes_count() -{ - cat < -#include -#include -#include - -HashMap, DefaultHash> error_codes_count; - -struct InitializeErrorCodesCount -{ - InitializeErrorCodesCount() - { -$(parse_for_error_codes_count) - } -} error_codes_count_initialize; -EOL -} - -function main() -{ - echo "// autogenerated by ${BASH_SOURCE[0]}" > "$TMP_FILE" - generate_errorCodeToName >> "$TMP_FILE" - generate_error_codes_count >> "$TMP_FILE" - - if [[ ! -e $ERROR_CODES_OUT_FILE ]]; then - cp -a "$TMP_FILE" "$ERROR_CODES_OUT_FILE" - fi - # update it only if it differs, to avoid costly recompilation - if ! diff -q "$TMP_FILE" "$ERROR_CODES_OUT_FILE"; then - cp -a "$TMP_FILE" "$ERROR_CODES_OUT_FILE" - fi -} -main "$@" diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index b528b3a276f..bdf89c983f1 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -53,8 +53,6 @@ endif() target_include_directories(clickhouse_functions SYSTEM PRIVATE ${SPARSEHASH_INCLUDE_DIR}) -add_dependencies(clickhouse_functions generate-error-codes) - if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL") diff --git a/src/Functions/errorCodeToName.cpp b/src/Functions/errorCodeToName.cpp index 8c9f49b2ac8..6591e46d338 100644 --- a/src/Functions/errorCodeToName.cpp +++ b/src/Functions/errorCodeToName.cpp @@ -5,9 +5,7 @@ #include #include #include -#include - -extern std::string_view errorCodeToName(int code); +#include namespace DB { @@ -48,7 +46,7 @@ public: for (size_t i = 0; i < input_rows_count; ++i) { const Int64 error_code = input_column.getInt(i); - std::string_view error_name = errorCodeToName(error_code); + std::string_view error_name = ErrorCodes::getName(error_code); col_res->insertData(error_name.data(), error_name.size()); } diff --git a/src/Storages/System/StorageSystemErrors.cpp b/src/Storages/System/StorageSystemErrors.cpp index 5ab6d02a78f..675bf7556de 100644 --- a/src/Storages/System/StorageSystemErrors.cpp +++ b/src/Storages/System/StorageSystemErrors.cpp @@ -1,20 +1,8 @@ -#include -#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -extern std::string_view errorCodeToName(int code); -extern HashMap, DefaultHash> error_codes_count; +#include +#include namespace DB { @@ -29,14 +17,23 @@ NamesAndTypesList StorageSystemErrors::getNamesAndTypes() } -void StorageSystemErrors::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const +void StorageSystemErrors::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - for (const auto & error_code_pair : error_codes_count) + for (size_t i = 0, end = ErrorCodes::end(); i < end; ++i) { - size_t col_num = 0; - res_columns[col_num++]->insert(errorCodeToName(error_code_pair.getKey())); - res_columns[col_num++]->insert(error_code_pair.getKey()); - res_columns[col_num++]->insert(uint64_t(error_code_pair.getMapped())); + UInt64 value = ErrorCodes::values[i]; + std::string_view name = ErrorCodes::getName(i); + + if (!name.size()) + continue; + + if (value || context.getSettingsRef().system_events_show_zero_values) + { + size_t col_num = 0; + res_columns[col_num++]->insert(name); + res_columns[col_num++]->insert(i); + res_columns[col_num++]->insert(value); + } } } diff --git a/tests/queries/0_stateless/01544_errorCodeToName.reference b/tests/queries/0_stateless/01544_errorCodeToName.reference index 4f0073384d9..ace588644e1 100644 --- a/tests/queries/0_stateless/01544_errorCodeToName.reference +++ b/tests/queries/0_stateless/01544_errorCodeToName.reference @@ -1,3 +1,4 @@ + OK UNSUPPORTED_METHOD diff --git a/tests/queries/0_stateless/01544_errorCodeToName.sql b/tests/queries/0_stateless/01544_errorCodeToName.sql index 07b46767b73..9e28ed1116c 100644 --- a/tests/queries/0_stateless/01544_errorCodeToName.sql +++ b/tests/queries/0_stateless/01544_errorCodeToName.sql @@ -1,3 +1,4 @@ SELECT errorCodeToName(toUInt32(-1)); +SELECT errorCodeToName(600); /* gap in error codes */ SELECT errorCodeToName(0); SELECT errorCodeToName(1); From 3bded85c4fcd185731a8577d0cd0a70b190d7022 Mon Sep 17 00:00:00 2001 From: ana-uvarova Date: Thu, 29 Oct 2020 15:43:18 +0300 Subject: [PATCH 083/314] Details --- docs/en/operations/system-tables/asynchronous_metric_log.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/asynchronous_metric_log.md b/docs/en/operations/system-tables/asynchronous_metric_log.md index e1eabdefa2f..5dcfca5fbda 100644 --- a/docs/en/operations/system-tables/asynchronous_metric_log.md +++ b/docs/en/operations/system-tables/asynchronous_metric_log.md @@ -33,7 +33,7 @@ SELECT * FROM system.asynchronous_metric_log LIMIT 10 **See Also** -- [system.asynchronous_metrics](../system-tables/asynchronous_metrics.md) — Contains metrics that are calculated periodically in the background. +- [system.asynchronous_metrics](../system-tables/asynchronous_metrics.md) — Contains metrics, calculated periodically in the background. - [system.metric_log](../system-tables/metric_log.md) — Contains history of metrics values from tables `system.metrics` and `system.events`, periodically flushed to disk. [Original article](https://clickhouse.tech/docs/en/operations/system_tables/asynchronous_metric_log) From 7e79769ae47e6802188c5221f1fe2992792f6a4a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Oct 2020 21:08:09 +0300 Subject: [PATCH 084/314] Fix readability-inconsistent-declaration-parameter-name in ErrorCodes.h --- src/Common/ErrorCodes.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ErrorCodes.h b/src/Common/ErrorCodes.h index 008cf018bd6..9afa4daae90 100644 --- a/src/Common/ErrorCodes.h +++ b/src/Common/ErrorCodes.h @@ -22,7 +22,7 @@ namespace ErrorCodes /// Get name of error_code by identifier. /// Returns statically allocated string. - std::string_view getName(ErrorCode event); + std::string_view getName(ErrorCode error_code); /// ErrorCode identifier -> current value of error_code. extern std::atomic values[]; From 04bff595d3639585a713249a9994f894f80117de Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Oct 2020 21:10:08 +0300 Subject: [PATCH 085/314] Make check-style happy (by not using DB::ErrorCodes:: in comments) --- src/Common/ErrorCodes.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 7b671427753..3b3f9af9d5e 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -524,7 +524,7 @@ M(CONDITIONAL_TREE_PARENT_NOT_FOUND, 2001) \ M(ILLEGAL_PROJECTION_MANIPULATOR, 2002) \ M(UNRECOGNIZED_ARGUMENTS, 2003) -/* See DB::ErrorCodes::END */ +/* See END */ namespace DB { From cb50886ef1436d100c0388f88ff4c50cae19a50d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Oct 2020 21:24:52 +0300 Subject: [PATCH 086/314] Fix ErrorCodes increment on error --- src/Common/Exception.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index c57726faa7c..0214fa7b065 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -46,8 +46,8 @@ Exception::Exception(const std::string & msg, int code) LOG_FATAL(&Poco::Logger::root(), "Logical error: '{}'.", msg); abort(); } - ErrorCodes::increment(code); #endif + ErrorCodes::increment(code); } Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc) From 0cd79de632047a3cbecd829616abb1456ca3a96e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Oct 2020 21:53:09 +0300 Subject: [PATCH 087/314] Eliminate extra strlen() in errorCodeToName() --- src/Common/ErrorCodes.cpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 3b3f9af9d5e..73b19aa4354 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -540,10 +540,10 @@ namespace ErrorCodes struct ErrorCodesNames { - const char * strings[END+1]; + std::string_view names[END+1]; ErrorCodesNames() { - #define M(NAME, VALUE) strings[VALUE] = #NAME; + #define M(NAME, VALUE) names[VALUE] = std::string_view(#NAME); APPLY_FOR_ERROR_CODES(M) #undef M } @@ -553,10 +553,7 @@ namespace ErrorCodes { if (error_code >= END) return std::string_view(); - const char * name = error_codes_names.strings[error_code]; - if (!name) - return std::string_view(); - return std::string_view(name); + return error_codes_names.names[error_code]; } ErrorCode end() { return END+1; } From 70e61362465a6b61ae34800ada213d710182b47b Mon Sep 17 00:00:00 2001 From: ana-uvarova Date: Thu, 29 Oct 2020 21:56:00 +0300 Subject: [PATCH 088/314] Russian --- docs/ru/operations/system-tables/asynchronous_metric_log.md | 1 + docs/ru/operations/system-tables/metric_log.md | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/docs/ru/operations/system-tables/asynchronous_metric_log.md b/docs/ru/operations/system-tables/asynchronous_metric_log.md index 719e965ebe4..eded0ba7b28 100644 --- a/docs/ru/operations/system-tables/asynchronous_metric_log.md +++ b/docs/ru/operations/system-tables/asynchronous_metric_log.md @@ -5,6 +5,7 @@ Столбцы: - `event_date` ([Date](../../sql-reference/data-types/date.md)) — дата события. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — время события. +- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — время события в секундах. - `name` ([String](../../sql-reference/data-types/string.md)) — название метрики. - `value` ([Float64](../../sql-reference/data-types/float.md)) — значение метрики. diff --git a/docs/ru/operations/system-tables/metric_log.md b/docs/ru/operations/system-tables/metric_log.md index 32ebe91dd91..fff5c57a113 100644 --- a/docs/ru/operations/system-tables/metric_log.md +++ b/docs/ru/operations/system-tables/metric_log.md @@ -1,6 +1,7 @@ # system.metric_log {#system_tables-metric_log} Содержит историю значений метрик из таблиц `system.metrics` и `system.events`, периодически сбрасываемую на диск. + Для включения сбора истории метрик в таблице `system.metric_log` создайте `/etc/clickhouse-server/config.d/metric_log.xml` следующего содержания: ``` xml @@ -14,6 +15,11 @@ ``` +Столбцы: +- `event_date` ([Date](../../sql-reference/data-types/date.md)) — дата события. +- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — время события. +- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — время события в секундах. + **Пример** ``` sql From ce9897bfd8054f8b382b8a69fe1747fbbddb6523 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 30 Oct 2020 00:07:40 +0300 Subject: [PATCH 089/314] Fix readability-container-size-empty in StorageSystemErrors.cpp --- src/Storages/System/StorageSystemErrors.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemErrors.cpp b/src/Storages/System/StorageSystemErrors.cpp index 675bf7556de..89df058900b 100644 --- a/src/Storages/System/StorageSystemErrors.cpp +++ b/src/Storages/System/StorageSystemErrors.cpp @@ -24,7 +24,7 @@ void StorageSystemErrors::fillData(MutableColumns & res_columns, const Context & UInt64 value = ErrorCodes::values[i]; std::string_view name = ErrorCodes::getName(i); - if (!name.size()) + if (name.empty()) continue; if (value || context.getSettingsRef().system_events_show_zero_values) From 98f701dbfcfdde171a3c3fedaffc28723a7fb6b4 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Fri, 30 Oct 2020 04:09:42 +0300 Subject: [PATCH 090/314] Docs in English --- docs/en/sql-reference/table-functions/null.md | 39 +++++++++++++++++++ 1 file changed, 39 insertions(+) create mode 100644 docs/en/sql-reference/table-functions/null.md diff --git a/docs/en/sql-reference/table-functions/null.md b/docs/en/sql-reference/table-functions/null.md new file mode 100644 index 00000000000..988ba18d709 --- /dev/null +++ b/docs/en/sql-reference/table-functions/null.md @@ -0,0 +1,39 @@ +--- +toc_priority: 53 +toc_title: null function +--- + +# null {#null-function} + +Accepts an inserted data of the specified structure and immediately drops it away. The function is used for convenience writing tests and demos. + +**Syntax** + +``` sql +null('structure') +``` + +**Parameter** + +- `structure` — A list of columns and column types. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +A table with the specified structure, which is dropped right after the query execution. + +**Example** + +Query with the `null` function: + +``` sql +INSERT INTO function null('x UInt64') SELECT * FROM numbers_mt(1000000000); +``` +can replace three queries: + +```sql +CREATE TABLE t (x UInt64) ENGINE = Null; +INSERT INTO t SELECT * FROM numbers_mt(1000000000); +DROP TABLE IF EXISTS t; +``` + +[Original article](https://clickhouse.tech/docs/en/sql-reference/table-functions/null/) From 3661769e1b4fef9f89be6b562c2bdd4595301ff8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 30 Oct 2020 08:55:40 +0300 Subject: [PATCH 091/314] Remove some absolute changes for errorCodeToName/system.errors --- src/Common/ya.make.in | 2 +- src/Functions/.gitignore | 1 - src/Functions/errorCodeToName.cpp | 4 ---- src/Functions/registerFunctionsMiscellaneous.cpp | 4 ---- src/Functions/ya.make | 1 + src/Functions/ya.make.in | 2 +- src/Storages/System/attachSystemTables.cpp | 4 ++-- 7 files changed, 5 insertions(+), 13 deletions(-) delete mode 100644 src/Functions/.gitignore diff --git a/src/Common/ya.make.in b/src/Common/ya.make.in index e0e3fd9a944..f8b7601e215 100644 --- a/src/Common/ya.make.in +++ b/src/Common/ya.make.in @@ -23,7 +23,7 @@ INCLUDE(${ARCADIA_ROOT}/clickhouse/cmake/yandex/ya.make.versions.inc) CFLAGS(-g0) SRCS( - + ) END() diff --git a/src/Functions/.gitignore b/src/Functions/.gitignore deleted file mode 100644 index 25db3625c77..00000000000 --- a/src/Functions/.gitignore +++ /dev/null @@ -1 +0,0 @@ -/errorCodes.generated.cpp diff --git a/src/Functions/errorCodeToName.cpp b/src/Functions/errorCodeToName.cpp index 6591e46d338..01fd5632288 100644 --- a/src/Functions/errorCodeToName.cpp +++ b/src/Functions/errorCodeToName.cpp @@ -1,5 +1,3 @@ -#if !defined(ARCADIA_BUILD) - #include #include #include @@ -61,5 +59,3 @@ void registerFunctionErrorCodeToName(FunctionFactory & factory) } } - -#endif diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 4a989831b48..86c06b47b1d 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -64,9 +64,7 @@ void registerFunctionCountDigits(FunctionFactory &); void registerFunctionGlobalVariable(FunctionFactory &); void registerFunctionHasThreadFuzzer(FunctionFactory &); void registerFunctionInitializeAggregation(FunctionFactory &); -#if !defined(ARCADIA_BUILD) void registerFunctionErrorCodeToName(FunctionFactory &); -#endif #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -131,9 +129,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionGlobalVariable(factory); registerFunctionHasThreadFuzzer(factory); registerFunctionInitializeAggregation(factory); -#if !defined(ARCADIA_BUILD) registerFunctionErrorCodeToName(factory); -#endif #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/src/Functions/ya.make b/src/Functions/ya.make index ed03f5175ab..4229190ba6c 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -156,6 +156,7 @@ SRCS( equals.cpp erfc.cpp erf.cpp + errorCodeToName.cpp evalMLMethod.cpp exp10.cpp exp2.cpp diff --git a/src/Functions/ya.make.in b/src/Functions/ya.make.in index e70658bb79d..2a66aa5553e 100644 --- a/src/Functions/ya.make.in +++ b/src/Functions/ya.make.in @@ -34,7 +34,7 @@ PEERDIR( CFLAGS(-g0) SRCS( - + ) END() diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 1ea24c1c4ba..210ed355e47 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -37,11 +37,11 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) #include #include #endif -#include #include #include #include @@ -108,10 +108,10 @@ void attachSystemTablesLocal(IDatabase & system_database) attach(system_database, "quotas_usage"); attach(system_database, "user_directories"); attach(system_database, "privileges"); + attach(system_database, "errors"); #if !defined(ARCADIA_BUILD) attach(system_database, "licenses"); attach(system_database, "time_zones"); - attach(system_database, "errors"); #endif #ifdef OS_LINUX attach(system_database, "stack_trace"); From 5481fcdf4211eec9751cd792e9d862c38328a942 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Fri, 30 Oct 2020 20:56:14 +0300 Subject: [PATCH 092/314] Fix script --- utils/make_changelog.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/make_changelog.py b/utils/make_changelog.py index 5a5c82e5ab6..9120267e3b0 100755 --- a/utils/make_changelog.py +++ b/utils/make_changelog.py @@ -262,7 +262,7 @@ def process_unknown_commits(commits, commits_info, users): # Returns False if the PR should not be mentioned changelog. def parse_one_pull_request(item): description = item['description'] - lines = [line for line in [x.strip() for x in description.split('\n') if description else []] if line] + lines = [line for line in [x.strip() for x in description.split('\n') if description] if line] lines = [re.sub(r'\s+', ' ', l) for l in lines] cat_pos = None From 1b60d1156c022d0ba879800f9612693c0f7a0cb7 Mon Sep 17 00:00:00 2001 From: ana-uvarova Date: Fri, 30 Oct 2020 21:06:29 +0300 Subject: [PATCH 093/314] MICRO back --- docs/ru/operations/system-tables/asynchronous_metric_log.md | 2 +- docs/ru/operations/system-tables/metric_log.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/system-tables/asynchronous_metric_log.md b/docs/ru/operations/system-tables/asynchronous_metric_log.md index eded0ba7b28..2fe617e48af 100644 --- a/docs/ru/operations/system-tables/asynchronous_metric_log.md +++ b/docs/ru/operations/system-tables/asynchronous_metric_log.md @@ -5,7 +5,7 @@ Столбцы: - `event_date` ([Date](../../sql-reference/data-types/date.md)) — дата события. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — время события. -- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — время события в секундах. +- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — время события в микросекундах. - `name` ([String](../../sql-reference/data-types/string.md)) — название метрики. - `value` ([Float64](../../sql-reference/data-types/float.md)) — значение метрики. diff --git a/docs/ru/operations/system-tables/metric_log.md b/docs/ru/operations/system-tables/metric_log.md index fff5c57a113..ad25e8d5cfd 100644 --- a/docs/ru/operations/system-tables/metric_log.md +++ b/docs/ru/operations/system-tables/metric_log.md @@ -18,7 +18,7 @@ Столбцы: - `event_date` ([Date](../../sql-reference/data-types/date.md)) — дата события. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — время события. -- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — время события в секундах. +- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — время события в микросекундах. **Пример** From 9828ed95504fb1a78380d47c8590a07bf8d4c884 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Fri, 30 Oct 2020 21:16:50 +0300 Subject: [PATCH 094/314] Update utils/make_changelog.py Co-authored-by: Azat Khuzhin --- utils/make_changelog.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/make_changelog.py b/utils/make_changelog.py index 9120267e3b0..4f703108d38 100755 --- a/utils/make_changelog.py +++ b/utils/make_changelog.py @@ -262,7 +262,7 @@ def process_unknown_commits(commits, commits_info, users): # Returns False if the PR should not be mentioned changelog. def parse_one_pull_request(item): description = item['description'] - lines = [line for line in [x.strip() for x in description.split('\n') if description] if line] + lines = [line for line in [x.strip() for x in description.split('\n')] if line] if description else [] lines = [re.sub(r'\s+', ' ', l) for l in lines] cat_pos = None From 2832255164979b72fcb1fabd4592a613abb74bd4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 30 Oct 2020 22:02:02 +0300 Subject: [PATCH 095/314] Increase asynchronous_metrics_update_period_s to avoid syncing MemoryTracking with RSS --- .../configs/asynchronous_metrics_update_period_s.xml | 4 ++++ .../test.py | 5 ++++- 2 files changed, 8 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_input_format_parallel_parsing_memory_tracking/configs/asynchronous_metrics_update_period_s.xml diff --git a/tests/integration/test_input_format_parallel_parsing_memory_tracking/configs/asynchronous_metrics_update_period_s.xml b/tests/integration/test_input_format_parallel_parsing_memory_tracking/configs/asynchronous_metrics_update_period_s.xml new file mode 100644 index 00000000000..ed131f41ede --- /dev/null +++ b/tests/integration/test_input_format_parallel_parsing_memory_tracking/configs/asynchronous_metrics_update_period_s.xml @@ -0,0 +1,4 @@ + + + 86400 + diff --git a/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py b/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py index 69c7a5821fd..e8866d3a235 100644 --- a/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py +++ b/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py @@ -8,7 +8,10 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance', main_configs=['configs/conf.xml']) +instance = cluster.add_instance('instance', main_configs=[ + 'configs/conf.xml', + 'configs/asynchronous_metrics_update_period_s.xml', +]) @pytest.fixture(scope='module', autouse=True) From 831208e2cf90db994fca0055241e496e21a8281c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 30 Oct 2020 22:02:02 +0300 Subject: [PATCH 096/314] Log the difference between process RSS and MemoryTracking metric --- src/Interpreters/AsynchronousMetrics.cpp | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index e1a9a820ebb..4e052349b6b 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -207,8 +207,22 @@ void AsynchronousMetrics::update() /// We must update the value of total_memory_tracker periodically. /// Otherwise it might be calculated incorrectly - it can include a "drift" of memory amount. /// See https://github.com/ClickHouse/ClickHouse/issues/10293 - total_memory_tracker.set(data.resident); - CurrentMetrics::set(CurrentMetrics::MemoryTracking, data.resident); + { + Int64 amount = total_memory_tracker.get(); + Int64 peak = total_memory_tracker.getPeak(); + Int64 new_peak = data.resident; + + LOG_DEBUG(&Poco::Logger::get("AsynchronousMetrics"), + "MemoryTracking: was {}, peak {}, will set to {} (RSS), difference: {}", + ReadableSize(amount), + ReadableSize(peak), + ReadableSize(new_peak), + ReadableSize(new_peak - peak) + ); + + total_memory_tracker.set(new_peak); + CurrentMetrics::set(CurrentMetrics::MemoryTracking, new_peak); + } } #endif From e07473f1ea35c60441ee5d1c48feecb6341c9a8e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 31 Oct 2020 11:29:17 +0300 Subject: [PATCH 097/314] Update comment for StorageSystemErrors --- src/Storages/System/StorageSystemErrors.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemErrors.h b/src/Storages/System/StorageSystemErrors.h index 3f2d3020bcd..569a7a998b7 100644 --- a/src/Storages/System/StorageSystemErrors.h +++ b/src/Storages/System/StorageSystemErrors.h @@ -10,8 +10,10 @@ namespace DB class Context; -/** Implements the `distribution_queue` system table, which allows you to view the INSERT queues for the Distributed tables. - */ +/** + * Implements the `errors` system table, which shows the error code and the number of times it happens + * (i.e. Exception with this code had been thrown). + */ class StorageSystemErrors final : public ext::shared_ptr_helper, public IStorageSystemOneBlock { friend struct ext::shared_ptr_helper; From c8f756707f751ffa6aeddf437900155f1b934b62 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 31 Oct 2020 11:31:59 +0300 Subject: [PATCH 098/314] Swap name and value in ErrorCodes definition --- src/Common/ErrorCodes.cpp | 1024 ++++++++++++++++++------------------- 1 file changed, 512 insertions(+), 512 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 73b19aa4354..968b847d686 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -12,518 +12,518 @@ */ #define APPLY_FOR_ERROR_CODES(M) \ - M(OK, 0) \ - M(UNSUPPORTED_METHOD, 1) \ - M(UNSUPPORTED_PARAMETER, 2) \ - M(UNEXPECTED_END_OF_FILE, 3) \ - M(EXPECTED_END_OF_FILE, 4) \ - M(CANNOT_PARSE_TEXT, 6) \ - M(INCORRECT_NUMBER_OF_COLUMNS, 7) \ - M(THERE_IS_NO_COLUMN, 8) \ - M(SIZES_OF_COLUMNS_DOESNT_MATCH, 9) \ - M(NOT_FOUND_COLUMN_IN_BLOCK, 10) \ - M(POSITION_OUT_OF_BOUND, 11) \ - M(PARAMETER_OUT_OF_BOUND, 12) \ - M(SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH, 13) \ - M(DUPLICATE_COLUMN, 15) \ - M(NO_SUCH_COLUMN_IN_TABLE, 16) \ - M(DELIMITER_IN_STRING_LITERAL_DOESNT_MATCH, 17) \ - M(CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN, 18) \ - M(SIZE_OF_FIXED_STRING_DOESNT_MATCH, 19) \ - M(NUMBER_OF_COLUMNS_DOESNT_MATCH, 20) \ - M(CANNOT_READ_ALL_DATA_FROM_TAB_SEPARATED_INPUT, 21) \ - M(CANNOT_PARSE_ALL_VALUE_FROM_TAB_SEPARATED_INPUT, 22) \ - M(CANNOT_READ_FROM_ISTREAM, 23) \ - M(CANNOT_WRITE_TO_OSTREAM, 24) \ - M(CANNOT_PARSE_ESCAPE_SEQUENCE, 25) \ - M(CANNOT_PARSE_QUOTED_STRING, 26) \ - M(CANNOT_PARSE_INPUT_ASSERTION_FAILED, 27) \ - M(CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, 28) \ - M(CANNOT_PRINT_INTEGER, 29) \ - M(CANNOT_READ_SIZE_OF_COMPRESSED_CHUNK, 30) \ - M(CANNOT_READ_COMPRESSED_CHUNK, 31) \ - M(ATTEMPT_TO_READ_AFTER_EOF, 32) \ - M(CANNOT_READ_ALL_DATA, 33) \ - M(TOO_MANY_ARGUMENTS_FOR_FUNCTION, 34) \ - M(TOO_FEW_ARGUMENTS_FOR_FUNCTION, 35) \ - M(BAD_ARGUMENTS, 36) \ - M(UNKNOWN_ELEMENT_IN_AST, 37) \ - M(CANNOT_PARSE_DATE, 38) \ - M(TOO_LARGE_SIZE_COMPRESSED, 39) \ - M(CHECKSUM_DOESNT_MATCH, 40) \ - M(CANNOT_PARSE_DATETIME, 41) \ - M(NUMBER_OF_ARGUMENTS_DOESNT_MATCH, 42) \ - M(ILLEGAL_TYPE_OF_ARGUMENT, 43) \ - M(ILLEGAL_COLUMN, 44) \ - M(ILLEGAL_NUMBER_OF_RESULT_COLUMNS, 45) \ - M(UNKNOWN_FUNCTION, 46) \ - M(UNKNOWN_IDENTIFIER, 47) \ - M(NOT_IMPLEMENTED, 48) \ - M(LOGICAL_ERROR, 49) \ - M(UNKNOWN_TYPE, 50) \ - M(EMPTY_LIST_OF_COLUMNS_QUERIED, 51) \ - M(COLUMN_QUERIED_MORE_THAN_ONCE, 52) \ - M(TYPE_MISMATCH, 53) \ - M(STORAGE_DOESNT_ALLOW_PARAMETERS, 54) \ - M(STORAGE_REQUIRES_PARAMETER, 55) \ - M(UNKNOWN_STORAGE, 56) \ - M(TABLE_ALREADY_EXISTS, 57) \ - M(TABLE_METADATA_ALREADY_EXISTS, 58) \ - M(ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, 59) \ - M(UNKNOWN_TABLE, 60) \ - M(ONLY_FILTER_COLUMN_IN_BLOCK, 61) \ - M(SYNTAX_ERROR, 62) \ - M(UNKNOWN_AGGREGATE_FUNCTION, 63) \ - M(CANNOT_READ_AGGREGATE_FUNCTION_FROM_TEXT, 64) \ - M(CANNOT_WRITE_AGGREGATE_FUNCTION_AS_TEXT, 65) \ - M(NOT_A_COLUMN, 66) \ - M(ILLEGAL_KEY_OF_AGGREGATION, 67) \ - M(CANNOT_GET_SIZE_OF_FIELD, 68) \ - M(ARGUMENT_OUT_OF_BOUND, 69) \ - M(CANNOT_CONVERT_TYPE, 70) \ - M(CANNOT_WRITE_AFTER_END_OF_BUFFER, 71) \ - M(CANNOT_PARSE_NUMBER, 72) \ - M(UNKNOWN_FORMAT, 73) \ - M(CANNOT_READ_FROM_FILE_DESCRIPTOR, 74) \ - M(CANNOT_WRITE_TO_FILE_DESCRIPTOR, 75) \ - M(CANNOT_OPEN_FILE, 76) \ - M(CANNOT_CLOSE_FILE, 77) \ - M(UNKNOWN_TYPE_OF_QUERY, 78) \ - M(INCORRECT_FILE_NAME, 79) \ - M(INCORRECT_QUERY, 80) \ - M(UNKNOWN_DATABASE, 81) \ - M(DATABASE_ALREADY_EXISTS, 82) \ - M(DIRECTORY_DOESNT_EXIST, 83) \ - M(DIRECTORY_ALREADY_EXISTS, 84) \ - M(FORMAT_IS_NOT_SUITABLE_FOR_INPUT, 85) \ - M(RECEIVED_ERROR_FROM_REMOTE_IO_SERVER, 86) \ - M(CANNOT_SEEK_THROUGH_FILE, 87) \ - M(CANNOT_TRUNCATE_FILE, 88) \ - M(UNKNOWN_COMPRESSION_METHOD, 89) \ - M(EMPTY_LIST_OF_COLUMNS_PASSED, 90) \ - M(SIZES_OF_MARKS_FILES_ARE_INCONSISTENT, 91) \ - M(EMPTY_DATA_PASSED, 92) \ - M(UNKNOWN_AGGREGATED_DATA_VARIANT, 93) \ - M(CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS, 94) \ - M(CANNOT_READ_FROM_SOCKET, 95) \ - M(CANNOT_WRITE_TO_SOCKET, 96) \ - M(CANNOT_READ_ALL_DATA_FROM_CHUNKED_INPUT, 97) \ - M(CANNOT_WRITE_TO_EMPTY_BLOCK_OUTPUT_STREAM, 98) \ - M(UNKNOWN_PACKET_FROM_CLIENT, 99) \ - M(UNKNOWN_PACKET_FROM_SERVER, 100) \ - M(UNEXPECTED_PACKET_FROM_CLIENT, 101) \ - M(UNEXPECTED_PACKET_FROM_SERVER, 102) \ - M(RECEIVED_DATA_FOR_WRONG_QUERY_ID, 103) \ - M(TOO_SMALL_BUFFER_SIZE, 104) \ - M(CANNOT_READ_HISTORY, 105) \ - M(CANNOT_APPEND_HISTORY, 106) \ - M(FILE_DOESNT_EXIST, 107) \ - M(NO_DATA_TO_INSERT, 108) \ - M(CANNOT_BLOCK_SIGNAL, 109) \ - M(CANNOT_UNBLOCK_SIGNAL, 110) \ - M(CANNOT_MANIPULATE_SIGSET, 111) \ - M(CANNOT_WAIT_FOR_SIGNAL, 112) \ - M(THERE_IS_NO_SESSION, 113) \ - M(CANNOT_CLOCK_GETTIME, 114) \ - M(UNKNOWN_SETTING, 115) \ - M(THERE_IS_NO_DEFAULT_VALUE, 116) \ - M(INCORRECT_DATA, 117) \ - M(ENGINE_REQUIRED, 119) \ - M(CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE, 120) \ - M(UNSUPPORTED_JOIN_KEYS, 121) \ - M(INCOMPATIBLE_COLUMNS, 122) \ - M(UNKNOWN_TYPE_OF_AST_NODE, 123) \ - M(INCORRECT_ELEMENT_OF_SET, 124) \ - M(INCORRECT_RESULT_OF_SCALAR_SUBQUERY, 125) \ - M(CANNOT_GET_RETURN_TYPE, 126) \ - M(ILLEGAL_INDEX, 127) \ - M(TOO_LARGE_ARRAY_SIZE, 128) \ - M(FUNCTION_IS_SPECIAL, 129) \ - M(CANNOT_READ_ARRAY_FROM_TEXT, 130) \ - M(TOO_LARGE_STRING_SIZE, 131) \ - M(AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS, 133) \ - M(PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS, 134) \ - M(ZERO_ARRAY_OR_TUPLE_INDEX, 135) \ - M(UNKNOWN_ELEMENT_IN_CONFIG, 137) \ - M(EXCESSIVE_ELEMENT_IN_CONFIG, 138) \ - M(NO_ELEMENTS_IN_CONFIG, 139) \ - M(ALL_REQUESTED_COLUMNS_ARE_MISSING, 140) \ - M(SAMPLING_NOT_SUPPORTED, 141) \ - M(NOT_FOUND_NODE, 142) \ - M(FOUND_MORE_THAN_ONE_NODE, 143) \ - M(FIRST_DATE_IS_BIGGER_THAN_LAST_DATE, 144) \ - M(UNKNOWN_OVERFLOW_MODE, 145) \ - M(QUERY_SECTION_DOESNT_MAKE_SENSE, 146) \ - M(NOT_FOUND_FUNCTION_ELEMENT_FOR_AGGREGATE, 147) \ - M(NOT_FOUND_RELATION_ELEMENT_FOR_CONDITION, 148) \ - M(NOT_FOUND_RHS_ELEMENT_FOR_CONDITION, 149) \ - M(EMPTY_LIST_OF_ATTRIBUTES_PASSED, 150) \ - M(INDEX_OF_COLUMN_IN_SORT_CLAUSE_IS_OUT_OF_RANGE, 151) \ - M(UNKNOWN_DIRECTION_OF_SORTING, 152) \ - M(ILLEGAL_DIVISION, 153) \ - M(AGGREGATE_FUNCTION_NOT_APPLICABLE, 154) \ - M(UNKNOWN_RELATION, 155) \ - M(DICTIONARIES_WAS_NOT_LOADED, 156) \ - M(ILLEGAL_OVERFLOW_MODE, 157) \ - M(TOO_MANY_ROWS, 158) \ - M(TIMEOUT_EXCEEDED, 159) \ - M(TOO_SLOW, 160) \ - M(TOO_MANY_COLUMNS, 161) \ - M(TOO_DEEP_SUBQUERIES, 162) \ - M(TOO_DEEP_PIPELINE, 163) \ - M(READONLY, 164) \ - M(TOO_MANY_TEMPORARY_COLUMNS, 165) \ - M(TOO_MANY_TEMPORARY_NON_CONST_COLUMNS, 166) \ - M(TOO_DEEP_AST, 167) \ - M(TOO_BIG_AST, 168) \ - M(BAD_TYPE_OF_FIELD, 169) \ - M(BAD_GET, 170) \ - M(CANNOT_CREATE_DIRECTORY, 172) \ - M(CANNOT_ALLOCATE_MEMORY, 173) \ - M(CYCLIC_ALIASES, 174) \ - M(CHUNK_NOT_FOUND, 176) \ - M(DUPLICATE_CHUNK_NAME, 177) \ - M(MULTIPLE_ALIASES_FOR_EXPRESSION, 178) \ - M(MULTIPLE_EXPRESSIONS_FOR_ALIAS, 179) \ - M(THERE_IS_NO_PROFILE, 180) \ - M(ILLEGAL_FINAL, 181) \ - M(ILLEGAL_PREWHERE, 182) \ - M(UNEXPECTED_EXPRESSION, 183) \ - M(ILLEGAL_AGGREGATION, 184) \ - M(UNSUPPORTED_MYISAM_BLOCK_TYPE, 185) \ - M(UNSUPPORTED_COLLATION_LOCALE, 186) \ - M(COLLATION_COMPARISON_FAILED, 187) \ - M(UNKNOWN_ACTION, 188) \ - M(TABLE_MUST_NOT_BE_CREATED_MANUALLY, 189) \ - M(SIZES_OF_ARRAYS_DOESNT_MATCH, 190) \ - M(SET_SIZE_LIMIT_EXCEEDED, 191) \ - M(UNKNOWN_USER, 192) \ - M(WRONG_PASSWORD, 193) \ - M(REQUIRED_PASSWORD, 194) \ - M(IP_ADDRESS_NOT_ALLOWED, 195) \ - M(UNKNOWN_ADDRESS_PATTERN_TYPE, 196) \ - M(SERVER_REVISION_IS_TOO_OLD, 197) \ - M(DNS_ERROR, 198) \ - M(UNKNOWN_QUOTA, 199) \ - M(QUOTA_DOESNT_ALLOW_KEYS, 200) \ - M(QUOTA_EXPIRED, 201) \ - M(TOO_MANY_SIMULTANEOUS_QUERIES, 202) \ - M(NO_FREE_CONNECTION, 203) \ - M(CANNOT_FSYNC, 204) \ - M(NESTED_TYPE_TOO_DEEP, 205) \ - M(ALIAS_REQUIRED, 206) \ - M(AMBIGUOUS_IDENTIFIER, 207) \ - M(EMPTY_NESTED_TABLE, 208) \ - M(SOCKET_TIMEOUT, 209) \ - M(NETWORK_ERROR, 210) \ - M(EMPTY_QUERY, 211) \ - M(UNKNOWN_LOAD_BALANCING, 212) \ - M(UNKNOWN_TOTALS_MODE, 213) \ - M(CANNOT_STATVFS, 214) \ - M(NOT_AN_AGGREGATE, 215) \ - M(QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING, 216) \ - M(CLIENT_HAS_CONNECTED_TO_WRONG_PORT, 217) \ - M(TABLE_IS_DROPPED, 218) \ - M(DATABASE_NOT_EMPTY, 219) \ - M(DUPLICATE_INTERSERVER_IO_ENDPOINT, 220) \ - M(NO_SUCH_INTERSERVER_IO_ENDPOINT, 221) \ - M(ADDING_REPLICA_TO_NON_EMPTY_TABLE, 222) \ - M(UNEXPECTED_AST_STRUCTURE, 223) \ - M(REPLICA_IS_ALREADY_ACTIVE, 224) \ - M(NO_ZOOKEEPER, 225) \ - M(NO_FILE_IN_DATA_PART, 226) \ - M(UNEXPECTED_FILE_IN_DATA_PART, 227) \ - M(BAD_SIZE_OF_FILE_IN_DATA_PART, 228) \ - M(QUERY_IS_TOO_LARGE, 229) \ - M(NOT_FOUND_EXPECTED_DATA_PART, 230) \ - M(TOO_MANY_UNEXPECTED_DATA_PARTS, 231) \ - M(NO_SUCH_DATA_PART, 232) \ - M(BAD_DATA_PART_NAME, 233) \ - M(NO_REPLICA_HAS_PART, 234) \ - M(DUPLICATE_DATA_PART, 235) \ - M(ABORTED, 236) \ - M(NO_REPLICA_NAME_GIVEN, 237) \ - M(FORMAT_VERSION_TOO_OLD, 238) \ - M(CANNOT_MUNMAP, 239) \ - M(CANNOT_MREMAP, 240) \ - M(MEMORY_LIMIT_EXCEEDED, 241) \ - M(TABLE_IS_READ_ONLY, 242) \ - M(NOT_ENOUGH_SPACE, 243) \ - M(UNEXPECTED_ZOOKEEPER_ERROR, 244) \ - M(CORRUPTED_DATA, 246) \ - M(INCORRECT_MARK, 247) \ - M(INVALID_PARTITION_VALUE, 248) \ - M(NOT_ENOUGH_BLOCK_NUMBERS, 250) \ - M(NO_SUCH_REPLICA, 251) \ - M(TOO_MANY_PARTS, 252) \ - M(REPLICA_IS_ALREADY_EXIST, 253) \ - M(NO_ACTIVE_REPLICAS, 254) \ - M(TOO_MANY_RETRIES_TO_FETCH_PARTS, 255) \ - M(PARTITION_ALREADY_EXISTS, 256) \ - M(PARTITION_DOESNT_EXIST, 257) \ - M(UNION_ALL_RESULT_STRUCTURES_MISMATCH, 258) \ - M(CLIENT_OUTPUT_FORMAT_SPECIFIED, 260) \ - M(UNKNOWN_BLOCK_INFO_FIELD, 261) \ - M(BAD_COLLATION, 262) \ - M(CANNOT_COMPILE_CODE, 263) \ - M(INCOMPATIBLE_TYPE_OF_JOIN, 264) \ - M(NO_AVAILABLE_REPLICA, 265) \ - M(MISMATCH_REPLICAS_DATA_SOURCES, 266) \ - M(STORAGE_DOESNT_SUPPORT_PARALLEL_REPLICAS, 267) \ - M(CPUID_ERROR, 268) \ - M(INFINITE_LOOP, 269) \ - M(CANNOT_COMPRESS, 270) \ - M(CANNOT_DECOMPRESS, 271) \ - M(CANNOT_IO_SUBMIT, 272) \ - M(CANNOT_IO_GETEVENTS, 273) \ - M(AIO_READ_ERROR, 274) \ - M(AIO_WRITE_ERROR, 275) \ - M(INDEX_NOT_USED, 277) \ - M(ALL_CONNECTION_TRIES_FAILED, 279) \ - M(NO_AVAILABLE_DATA, 280) \ - M(DICTIONARY_IS_EMPTY, 281) \ - M(INCORRECT_INDEX, 282) \ - M(UNKNOWN_DISTRIBUTED_PRODUCT_MODE, 283) \ - M(WRONG_GLOBAL_SUBQUERY, 284) \ - M(TOO_FEW_LIVE_REPLICAS, 285) \ - M(UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE, 286) \ - M(UNKNOWN_FORMAT_VERSION, 287) \ - M(DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED, 288) \ - M(REPLICA_IS_NOT_IN_QUORUM, 289) \ - M(LIMIT_EXCEEDED, 290) \ - M(DATABASE_ACCESS_DENIED, 291) \ - M(MONGODB_CANNOT_AUTHENTICATE, 293) \ - M(INVALID_BLOCK_EXTRA_INFO, 294) \ - M(RECEIVED_EMPTY_DATA, 295) \ - M(NO_REMOTE_SHARD_FOUND, 296) \ - M(SHARD_HAS_NO_CONNECTIONS, 297) \ - M(CANNOT_PIPE, 298) \ - M(CANNOT_FORK, 299) \ - M(CANNOT_DLSYM, 300) \ - M(CANNOT_CREATE_CHILD_PROCESS, 301) \ - M(CHILD_WAS_NOT_EXITED_NORMALLY, 302) \ - M(CANNOT_SELECT, 303) \ - M(CANNOT_WAITPID, 304) \ - M(TABLE_WAS_NOT_DROPPED, 305) \ - M(TOO_DEEP_RECURSION, 306) \ - M(TOO_MANY_BYTES, 307) \ - M(UNEXPECTED_NODE_IN_ZOOKEEPER, 308) \ - M(FUNCTION_CANNOT_HAVE_PARAMETERS, 309) \ - M(INVALID_SHARD_WEIGHT, 317) \ - M(INVALID_CONFIG_PARAMETER, 318) \ - M(UNKNOWN_STATUS_OF_INSERT, 319) \ - M(VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, 321) \ - M(BARRIER_TIMEOUT, 335) \ - M(UNKNOWN_DATABASE_ENGINE, 336) \ - M(DDL_GUARD_IS_ACTIVE, 337) \ - M(UNFINISHED, 341) \ - M(METADATA_MISMATCH, 342) \ - M(SUPPORT_IS_DISABLED, 344) \ - M(TABLE_DIFFERS_TOO_MUCH, 345) \ - M(CANNOT_CONVERT_CHARSET, 346) \ - M(CANNOT_LOAD_CONFIG, 347) \ - M(CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN, 349) \ - M(INCOMPATIBLE_SOURCE_TABLES, 350) \ - M(AMBIGUOUS_TABLE_NAME, 351) \ - M(AMBIGUOUS_COLUMN_NAME, 352) \ - M(INDEX_OF_POSITIONAL_ARGUMENT_IS_OUT_OF_RANGE, 353) \ - M(ZLIB_INFLATE_FAILED, 354) \ - M(ZLIB_DEFLATE_FAILED, 355) \ - M(BAD_LAMBDA, 356) \ - M(RESERVED_IDENTIFIER_NAME, 357) \ - M(INTO_OUTFILE_NOT_ALLOWED, 358) \ - M(TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT, 359) \ - M(CANNOT_CREATE_CHARSET_CONVERTER, 360) \ - M(SEEK_POSITION_OUT_OF_BOUND, 361) \ - M(CURRENT_WRITE_BUFFER_IS_EXHAUSTED, 362) \ - M(CANNOT_CREATE_IO_BUFFER, 363) \ - M(RECEIVED_ERROR_TOO_MANY_REQUESTS, 364) \ - M(SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT, 366) \ - M(TOO_MANY_FETCHES, 367) \ - M(ALL_REPLICAS_ARE_STALE, 369) \ - M(DATA_TYPE_CANNOT_BE_USED_IN_TABLES, 370) \ - M(INCONSISTENT_CLUSTER_DEFINITION, 371) \ - M(SESSION_NOT_FOUND, 372) \ - M(SESSION_IS_LOCKED, 373) \ - M(INVALID_SESSION_TIMEOUT, 374) \ - M(CANNOT_DLOPEN, 375) \ - M(CANNOT_PARSE_UUID, 376) \ - M(ILLEGAL_SYNTAX_FOR_DATA_TYPE, 377) \ - M(DATA_TYPE_CANNOT_HAVE_ARGUMENTS, 378) \ - M(UNKNOWN_STATUS_OF_DISTRIBUTED_DDL_TASK, 379) \ - M(CANNOT_KILL, 380) \ - M(HTTP_LENGTH_REQUIRED, 381) \ - M(CANNOT_LOAD_CATBOOST_MODEL, 382) \ - M(CANNOT_APPLY_CATBOOST_MODEL, 383) \ - M(PART_IS_TEMPORARILY_LOCKED, 384) \ - M(MULTIPLE_STREAMS_REQUIRED, 385) \ - M(NO_COMMON_TYPE, 386) \ - M(DICTIONARY_ALREADY_EXISTS, 387) \ - M(CANNOT_ASSIGN_OPTIMIZE, 388) \ - M(INSERT_WAS_DEDUPLICATED, 389) \ - M(CANNOT_GET_CREATE_TABLE_QUERY, 390) \ - M(EXTERNAL_LIBRARY_ERROR, 391) \ - M(QUERY_IS_PROHIBITED, 392) \ - M(THERE_IS_NO_QUERY, 393) \ - M(QUERY_WAS_CANCELLED, 394) \ - M(FUNCTION_THROW_IF_VALUE_IS_NON_ZERO, 395) \ - M(TOO_MANY_ROWS_OR_BYTES, 396) \ - M(QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, 397) \ - M(UNKNOWN_MUTATION_COMMAND, 398) \ - M(FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT, 399) \ - M(CANNOT_STAT, 400) \ - M(FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, 401) \ - M(CANNOT_IOSETUP, 402) \ - M(INVALID_JOIN_ON_EXPRESSION, 403) \ - M(BAD_ODBC_CONNECTION_STRING, 404) \ - M(PARTITION_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT, 405) \ - M(TOP_AND_LIMIT_TOGETHER, 406) \ - M(DECIMAL_OVERFLOW, 407) \ - M(BAD_REQUEST_PARAMETER, 408) \ - M(EXTERNAL_EXECUTABLE_NOT_FOUND, 409) \ - M(EXTERNAL_SERVER_IS_NOT_RESPONDING, 410) \ - M(PTHREAD_ERROR, 411) \ - M(NETLINK_ERROR, 412) \ - M(CANNOT_SET_SIGNAL_HANDLER, 413) \ - M(ALL_REPLICAS_LOST, 415) \ - M(REPLICA_STATUS_CHANGED, 416) \ - M(EXPECTED_ALL_OR_ANY, 417) \ - M(UNKNOWN_JOIN, 418) \ - M(MULTIPLE_ASSIGNMENTS_TO_COLUMN, 419) \ - M(CANNOT_UPDATE_COLUMN, 420) \ - M(CANNOT_ADD_DIFFERENT_AGGREGATE_STATES, 421) \ - M(UNSUPPORTED_URI_SCHEME, 422) \ - M(CANNOT_GETTIMEOFDAY, 423) \ - M(CANNOT_LINK, 424) \ - M(SYSTEM_ERROR, 425) \ - M(CANNOT_COMPILE_REGEXP, 427) \ - M(UNKNOWN_LOG_LEVEL, 428) \ - M(FAILED_TO_GETPWUID, 429) \ - M(MISMATCHING_USERS_FOR_PROCESS_AND_DATA, 430) \ - M(ILLEGAL_SYNTAX_FOR_CODEC_TYPE, 431) \ - M(UNKNOWN_CODEC, 432) \ - M(ILLEGAL_CODEC_PARAMETER, 433) \ - M(CANNOT_PARSE_PROTOBUF_SCHEMA, 434) \ - M(NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD, 435) \ - M(PROTOBUF_BAD_CAST, 436) \ - M(PROTOBUF_FIELD_NOT_REPEATED, 437) \ - M(DATA_TYPE_CANNOT_BE_PROMOTED, 438) \ - M(CANNOT_SCHEDULE_TASK, 439) \ - M(INVALID_LIMIT_EXPRESSION, 440) \ - M(CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING, 441) \ - M(BAD_DATABASE_FOR_TEMPORARY_TABLE, 442) \ - M(NO_COMMON_COLUMNS_WITH_PROTOBUF_SCHEMA, 443) \ - M(UNKNOWN_PROTOBUF_FORMAT, 444) \ - M(CANNOT_MPROTECT, 445) \ - M(FUNCTION_NOT_ALLOWED, 446) \ - M(HYPERSCAN_CANNOT_SCAN_TEXT, 447) \ - M(BROTLI_READ_FAILED, 448) \ - M(BROTLI_WRITE_FAILED, 449) \ - M(BAD_TTL_EXPRESSION, 450) \ - M(BAD_TTL_FILE, 451) \ - M(SETTING_CONSTRAINT_VIOLATION, 452) \ - M(MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES, 453) \ - M(OPENSSL_ERROR, 454) \ - M(SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY, 455) \ - M(UNKNOWN_QUERY_PARAMETER, 456) \ - M(BAD_QUERY_PARAMETER, 457) \ - M(CANNOT_UNLINK, 458) \ - M(CANNOT_SET_THREAD_PRIORITY, 459) \ - M(CANNOT_CREATE_TIMER, 460) \ - M(CANNOT_SET_TIMER_PERIOD, 461) \ - M(CANNOT_DELETE_TIMER, 462) \ - M(CANNOT_FCNTL, 463) \ - M(CANNOT_PARSE_ELF, 464) \ - M(CANNOT_PARSE_DWARF, 465) \ - M(INSECURE_PATH, 466) \ - M(CANNOT_PARSE_BOOL, 467) \ - M(CANNOT_PTHREAD_ATTR, 468) \ - M(VIOLATED_CONSTRAINT, 469) \ - M(QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW, 470) \ - M(INVALID_SETTING_VALUE, 471) \ - M(READONLY_SETTING, 472) \ - M(DEADLOCK_AVOIDED, 473) \ - M(INVALID_TEMPLATE_FORMAT, 474) \ - M(INVALID_WITH_FILL_EXPRESSION, 475) \ - M(WITH_TIES_WITHOUT_ORDER_BY, 476) \ - M(INVALID_USAGE_OF_INPUT, 477) \ - M(UNKNOWN_POLICY, 478) \ - M(UNKNOWN_DISK, 479) \ - M(UNKNOWN_PROTOCOL, 480) \ - M(PATH_ACCESS_DENIED, 481) \ - M(DICTIONARY_ACCESS_DENIED, 482) \ - M(TOO_MANY_REDIRECTS, 483) \ - M(INTERNAL_REDIS_ERROR, 484) \ - M(SCALAR_ALREADY_EXISTS, 485) \ - M(CANNOT_GET_CREATE_DICTIONARY_QUERY, 487) \ - M(UNKNOWN_DICTIONARY, 488) \ - M(INCORRECT_DICTIONARY_DEFINITION, 489) \ - M(CANNOT_FORMAT_DATETIME, 490) \ - M(UNACCEPTABLE_URL, 491) \ - M(ACCESS_ENTITY_NOT_FOUND, 492) \ - M(ACCESS_ENTITY_ALREADY_EXISTS, 493) \ - M(ACCESS_ENTITY_FOUND_DUPLICATES, 494) \ - M(ACCESS_STORAGE_READONLY, 495) \ - M(QUOTA_REQUIRES_CLIENT_KEY, 496) \ - M(ACCESS_DENIED, 497) \ - M(LIMIT_BY_WITH_TIES_IS_NOT_SUPPORTED, 498) \ - M(S3_ERROR, 499) \ - M(CANNOT_CREATE_DATABASE, 501) \ - M(CANNOT_SIGQUEUE, 502) \ - M(AGGREGATE_FUNCTION_THROW, 503) \ - M(FILE_ALREADY_EXISTS, 504) \ - M(CANNOT_DELETE_DIRECTORY, 505) \ - M(UNEXPECTED_ERROR_CODE, 506) \ - M(UNABLE_TO_SKIP_UNUSED_SHARDS, 507) \ - M(UNKNOWN_ACCESS_TYPE, 508) \ - M(INVALID_GRANT, 509) \ - M(CACHE_DICTIONARY_UPDATE_FAIL, 510) \ - M(UNKNOWN_ROLE, 511) \ - M(SET_NON_GRANTED_ROLE, 512) \ - M(UNKNOWN_PART_TYPE, 513) \ - M(ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND, 514) \ - M(INCORRECT_ACCESS_ENTITY_DEFINITION, 515) \ - M(AUTHENTICATION_FAILED, 516) \ - M(CANNOT_ASSIGN_ALTER, 517) \ - M(CANNOT_COMMIT_OFFSET, 518) \ - M(NO_REMOTE_SHARD_AVAILABLE, 519) \ - M(CANNOT_DETACH_DICTIONARY_AS_TABLE, 520) \ - M(ATOMIC_RENAME_FAIL, 521) \ - M(UNKNOWN_ROW_POLICY, 523) \ - M(ALTER_OF_COLUMN_IS_FORBIDDEN, 524) \ - M(INCORRECT_DISK_INDEX, 525) \ - M(UNKNOWN_VOLUME_TYPE, 526) \ - M(NO_SUITABLE_FUNCTION_IMPLEMENTATION, 527) \ - M(CASSANDRA_INTERNAL_ERROR, 528) \ - M(NOT_A_LEADER, 529) \ - M(CANNOT_CONNECT_RABBITMQ, 530) \ - M(CANNOT_FSTAT, 531) \ - M(LDAP_ERROR, 532) \ - M(INCONSISTENT_RESERVATIONS, 533) \ - M(NO_RESERVATIONS_PROVIDED, 534) \ - M(UNKNOWN_RAID_TYPE, 535) \ - M(CANNOT_RESTORE_FROM_FIELD_DUMP, 536) \ - M(ILLEGAL_MYSQL_VARIABLE, 537) \ - M(MYSQL_SYNTAX_ERROR, 538) \ - M(CANNOT_BIND_RABBITMQ_EXCHANGE, 539) \ - M(CANNOT_DECLARE_RABBITMQ_EXCHANGE, 540) \ - M(CANNOT_CREATE_RABBITMQ_QUEUE_BINDING, 541) \ - M(CANNOT_REMOVE_RABBITMQ_EXCHANGE, 542) \ - M(UNKNOWN_MYSQL_DATATYPES_SUPPORT_LEVEL, 543) \ - M(ROW_AND_ROWS_TOGETHER, 544) \ - M(FIRST_AND_NEXT_TOGETHER, 545) \ - M(NO_ROW_DELIMITER, 546) \ - M(INVALID_RAID_TYPE, 547) \ - M(UNKNOWN_VOLUME, 548) \ + M(0, OK) \ + M(1, UNSUPPORTED_METHOD) \ + M(2, UNSUPPORTED_PARAMETER) \ + M(3, UNEXPECTED_END_OF_FILE) \ + M(4, EXPECTED_END_OF_FILE) \ + M(6, CANNOT_PARSE_TEXT) \ + M(7, INCORRECT_NUMBER_OF_COLUMNS) \ + M(8, THERE_IS_NO_COLUMN) \ + M(9, SIZES_OF_COLUMNS_DOESNT_MATCH) \ + M(10, NOT_FOUND_COLUMN_IN_BLOCK) \ + M(11, POSITION_OUT_OF_BOUND) \ + M(12, PARAMETER_OUT_OF_BOUND) \ + M(13, SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH) \ + M(15, DUPLICATE_COLUMN) \ + M(16, NO_SUCH_COLUMN_IN_TABLE) \ + M(17, DELIMITER_IN_STRING_LITERAL_DOESNT_MATCH) \ + M(18, CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN) \ + M(19, SIZE_OF_FIXED_STRING_DOESNT_MATCH) \ + M(20, NUMBER_OF_COLUMNS_DOESNT_MATCH) \ + M(21, CANNOT_READ_ALL_DATA_FROM_TAB_SEPARATED_INPUT) \ + M(22, CANNOT_PARSE_ALL_VALUE_FROM_TAB_SEPARATED_INPUT) \ + M(23, CANNOT_READ_FROM_ISTREAM) \ + M(24, CANNOT_WRITE_TO_OSTREAM) \ + M(25, CANNOT_PARSE_ESCAPE_SEQUENCE) \ + M(26, CANNOT_PARSE_QUOTED_STRING) \ + M(27, CANNOT_PARSE_INPUT_ASSERTION_FAILED) \ + M(28, CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER) \ + M(29, CANNOT_PRINT_INTEGER) \ + M(30, CANNOT_READ_SIZE_OF_COMPRESSED_CHUNK) \ + M(31, CANNOT_READ_COMPRESSED_CHUNK) \ + M(32, ATTEMPT_TO_READ_AFTER_EOF) \ + M(33, CANNOT_READ_ALL_DATA) \ + M(34, TOO_MANY_ARGUMENTS_FOR_FUNCTION) \ + M(35, TOO_FEW_ARGUMENTS_FOR_FUNCTION) \ + M(36, BAD_ARGUMENTS) \ + M(37, UNKNOWN_ELEMENT_IN_AST) \ + M(38, CANNOT_PARSE_DATE) \ + M(39, TOO_LARGE_SIZE_COMPRESSED) \ + M(40, CHECKSUM_DOESNT_MATCH) \ + M(41, CANNOT_PARSE_DATETIME) \ + M(42, NUMBER_OF_ARGUMENTS_DOESNT_MATCH) \ + M(43, ILLEGAL_TYPE_OF_ARGUMENT) \ + M(44, ILLEGAL_COLUMN) \ + M(45, ILLEGAL_NUMBER_OF_RESULT_COLUMNS) \ + M(46, UNKNOWN_FUNCTION) \ + M(47, UNKNOWN_IDENTIFIER) \ + M(48, NOT_IMPLEMENTED) \ + M(49, LOGICAL_ERROR) \ + M(50, UNKNOWN_TYPE) \ + M(51, EMPTY_LIST_OF_COLUMNS_QUERIED) \ + M(52, COLUMN_QUERIED_MORE_THAN_ONCE) \ + M(53, TYPE_MISMATCH) \ + M(54, STORAGE_DOESNT_ALLOW_PARAMETERS) \ + M(55, STORAGE_REQUIRES_PARAMETER) \ + M(56, UNKNOWN_STORAGE) \ + M(57, TABLE_ALREADY_EXISTS) \ + M(58, TABLE_METADATA_ALREADY_EXISTS) \ + M(59, ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER) \ + M(60, UNKNOWN_TABLE) \ + M(61, ONLY_FILTER_COLUMN_IN_BLOCK) \ + M(62, SYNTAX_ERROR) \ + M(63, UNKNOWN_AGGREGATE_FUNCTION) \ + M(64, CANNOT_READ_AGGREGATE_FUNCTION_FROM_TEXT) \ + M(65, CANNOT_WRITE_AGGREGATE_FUNCTION_AS_TEXT) \ + M(66, NOT_A_COLUMN) \ + M(67, ILLEGAL_KEY_OF_AGGREGATION) \ + M(68, CANNOT_GET_SIZE_OF_FIELD) \ + M(69, ARGUMENT_OUT_OF_BOUND) \ + M(70, CANNOT_CONVERT_TYPE) \ + M(71, CANNOT_WRITE_AFTER_END_OF_BUFFER) \ + M(72, CANNOT_PARSE_NUMBER) \ + M(73, UNKNOWN_FORMAT) \ + M(74, CANNOT_READ_FROM_FILE_DESCRIPTOR) \ + M(75, CANNOT_WRITE_TO_FILE_DESCRIPTOR) \ + M(76, CANNOT_OPEN_FILE) \ + M(77, CANNOT_CLOSE_FILE) \ + M(78, UNKNOWN_TYPE_OF_QUERY) \ + M(79, INCORRECT_FILE_NAME) \ + M(80, INCORRECT_QUERY) \ + M(81, UNKNOWN_DATABASE) \ + M(82, DATABASE_ALREADY_EXISTS) \ + M(83, DIRECTORY_DOESNT_EXIST) \ + M(84, DIRECTORY_ALREADY_EXISTS) \ + M(85, FORMAT_IS_NOT_SUITABLE_FOR_INPUT) \ + M(86, RECEIVED_ERROR_FROM_REMOTE_IO_SERVER) \ + M(87, CANNOT_SEEK_THROUGH_FILE) \ + M(88, CANNOT_TRUNCATE_FILE) \ + M(89, UNKNOWN_COMPRESSION_METHOD) \ + M(90, EMPTY_LIST_OF_COLUMNS_PASSED) \ + M(91, SIZES_OF_MARKS_FILES_ARE_INCONSISTENT) \ + M(92, EMPTY_DATA_PASSED) \ + M(93, UNKNOWN_AGGREGATED_DATA_VARIANT) \ + M(94, CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS) \ + M(95, CANNOT_READ_FROM_SOCKET) \ + M(96, CANNOT_WRITE_TO_SOCKET) \ + M(97, CANNOT_READ_ALL_DATA_FROM_CHUNKED_INPUT) \ + M(98, CANNOT_WRITE_TO_EMPTY_BLOCK_OUTPUT_STREAM) \ + M(99, UNKNOWN_PACKET_FROM_CLIENT) \ + M(100, UNKNOWN_PACKET_FROM_SERVER) \ + M(101, UNEXPECTED_PACKET_FROM_CLIENT) \ + M(102, UNEXPECTED_PACKET_FROM_SERVER) \ + M(103, RECEIVED_DATA_FOR_WRONG_QUERY_ID) \ + M(104, TOO_SMALL_BUFFER_SIZE) \ + M(105, CANNOT_READ_HISTORY) \ + M(106, CANNOT_APPEND_HISTORY) \ + M(107, FILE_DOESNT_EXIST) \ + M(108, NO_DATA_TO_INSERT) \ + M(109, CANNOT_BLOCK_SIGNAL) \ + M(110, CANNOT_UNBLOCK_SIGNAL) \ + M(111, CANNOT_MANIPULATE_SIGSET) \ + M(112, CANNOT_WAIT_FOR_SIGNAL) \ + M(113, THERE_IS_NO_SESSION) \ + M(114, CANNOT_CLOCK_GETTIME) \ + M(115, UNKNOWN_SETTING) \ + M(116, THERE_IS_NO_DEFAULT_VALUE) \ + M(117, INCORRECT_DATA) \ + M(119, ENGINE_REQUIRED) \ + M(120, CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE) \ + M(121, UNSUPPORTED_JOIN_KEYS) \ + M(122, INCOMPATIBLE_COLUMNS) \ + M(123, UNKNOWN_TYPE_OF_AST_NODE) \ + M(124, INCORRECT_ELEMENT_OF_SET) \ + M(125, INCORRECT_RESULT_OF_SCALAR_SUBQUERY) \ + M(126, CANNOT_GET_RETURN_TYPE) \ + M(127, ILLEGAL_INDEX) \ + M(128, TOO_LARGE_ARRAY_SIZE) \ + M(129, FUNCTION_IS_SPECIAL) \ + M(130, CANNOT_READ_ARRAY_FROM_TEXT) \ + M(131, TOO_LARGE_STRING_SIZE) \ + M(133, AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS) \ + M(134, PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS) \ + M(135, ZERO_ARRAY_OR_TUPLE_INDEX) \ + M(137, UNKNOWN_ELEMENT_IN_CONFIG) \ + M(138, EXCESSIVE_ELEMENT_IN_CONFIG) \ + M(139, NO_ELEMENTS_IN_CONFIG) \ + M(140, ALL_REQUESTED_COLUMNS_ARE_MISSING) \ + M(141, SAMPLING_NOT_SUPPORTED) \ + M(142, NOT_FOUND_NODE) \ + M(143, FOUND_MORE_THAN_ONE_NODE) \ + M(144, FIRST_DATE_IS_BIGGER_THAN_LAST_DATE) \ + M(145, UNKNOWN_OVERFLOW_MODE) \ + M(146, QUERY_SECTION_DOESNT_MAKE_SENSE) \ + M(147, NOT_FOUND_FUNCTION_ELEMENT_FOR_AGGREGATE) \ + M(148, NOT_FOUND_RELATION_ELEMENT_FOR_CONDITION) \ + M(149, NOT_FOUND_RHS_ELEMENT_FOR_CONDITION) \ + M(150, EMPTY_LIST_OF_ATTRIBUTES_PASSED) \ + M(151, INDEX_OF_COLUMN_IN_SORT_CLAUSE_IS_OUT_OF_RANGE) \ + M(152, UNKNOWN_DIRECTION_OF_SORTING) \ + M(153, ILLEGAL_DIVISION) \ + M(154, AGGREGATE_FUNCTION_NOT_APPLICABLE) \ + M(155, UNKNOWN_RELATION) \ + M(156, DICTIONARIES_WAS_NOT_LOADED) \ + M(157, ILLEGAL_OVERFLOW_MODE) \ + M(158, TOO_MANY_ROWS) \ + M(159, TIMEOUT_EXCEEDED) \ + M(160, TOO_SLOW) \ + M(161, TOO_MANY_COLUMNS) \ + M(162, TOO_DEEP_SUBQUERIES) \ + M(163, TOO_DEEP_PIPELINE) \ + M(164, READONLY) \ + M(165, TOO_MANY_TEMPORARY_COLUMNS) \ + M(166, TOO_MANY_TEMPORARY_NON_CONST_COLUMNS) \ + M(167, TOO_DEEP_AST) \ + M(168, TOO_BIG_AST) \ + M(169, BAD_TYPE_OF_FIELD) \ + M(170, BAD_GET) \ + M(172, CANNOT_CREATE_DIRECTORY) \ + M(173, CANNOT_ALLOCATE_MEMORY) \ + M(174, CYCLIC_ALIASES) \ + M(176, CHUNK_NOT_FOUND) \ + M(177, DUPLICATE_CHUNK_NAME) \ + M(178, MULTIPLE_ALIASES_FOR_EXPRESSION) \ + M(179, MULTIPLE_EXPRESSIONS_FOR_ALIAS) \ + M(180, THERE_IS_NO_PROFILE) \ + M(181, ILLEGAL_FINAL) \ + M(182, ILLEGAL_PREWHERE) \ + M(183, UNEXPECTED_EXPRESSION) \ + M(184, ILLEGAL_AGGREGATION) \ + M(185, UNSUPPORTED_MYISAM_BLOCK_TYPE) \ + M(186, UNSUPPORTED_COLLATION_LOCALE) \ + M(187, COLLATION_COMPARISON_FAILED) \ + M(188, UNKNOWN_ACTION) \ + M(189, TABLE_MUST_NOT_BE_CREATED_MANUALLY) \ + M(190, SIZES_OF_ARRAYS_DOESNT_MATCH) \ + M(191, SET_SIZE_LIMIT_EXCEEDED) \ + M(192, UNKNOWN_USER) \ + M(193, WRONG_PASSWORD) \ + M(194, REQUIRED_PASSWORD) \ + M(195, IP_ADDRESS_NOT_ALLOWED) \ + M(196, UNKNOWN_ADDRESS_PATTERN_TYPE) \ + M(197, SERVER_REVISION_IS_TOO_OLD) \ + M(198, DNS_ERROR) \ + M(199, UNKNOWN_QUOTA) \ + M(200, QUOTA_DOESNT_ALLOW_KEYS) \ + M(201, QUOTA_EXPIRED) \ + M(202, TOO_MANY_SIMULTANEOUS_QUERIES) \ + M(203, NO_FREE_CONNECTION) \ + M(204, CANNOT_FSYNC) \ + M(205, NESTED_TYPE_TOO_DEEP) \ + M(206, ALIAS_REQUIRED) \ + M(207, AMBIGUOUS_IDENTIFIER) \ + M(208, EMPTY_NESTED_TABLE) \ + M(209, SOCKET_TIMEOUT) \ + M(210, NETWORK_ERROR) \ + M(211, EMPTY_QUERY) \ + M(212, UNKNOWN_LOAD_BALANCING) \ + M(213, UNKNOWN_TOTALS_MODE) \ + M(214, CANNOT_STATVFS) \ + M(215, NOT_AN_AGGREGATE) \ + M(216, QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING) \ + M(217, CLIENT_HAS_CONNECTED_TO_WRONG_PORT) \ + M(218, TABLE_IS_DROPPED) \ + M(219, DATABASE_NOT_EMPTY) \ + M(220, DUPLICATE_INTERSERVER_IO_ENDPOINT) \ + M(221, NO_SUCH_INTERSERVER_IO_ENDPOINT) \ + M(222, ADDING_REPLICA_TO_NON_EMPTY_TABLE) \ + M(223, UNEXPECTED_AST_STRUCTURE) \ + M(224, REPLICA_IS_ALREADY_ACTIVE) \ + M(225, NO_ZOOKEEPER) \ + M(226, NO_FILE_IN_DATA_PART) \ + M(227, UNEXPECTED_FILE_IN_DATA_PART) \ + M(228, BAD_SIZE_OF_FILE_IN_DATA_PART) \ + M(229, QUERY_IS_TOO_LARGE) \ + M(230, NOT_FOUND_EXPECTED_DATA_PART) \ + M(231, TOO_MANY_UNEXPECTED_DATA_PARTS) \ + M(232, NO_SUCH_DATA_PART) \ + M(233, BAD_DATA_PART_NAME) \ + M(234, NO_REPLICA_HAS_PART) \ + M(235, DUPLICATE_DATA_PART) \ + M(236, ABORTED) \ + M(237, NO_REPLICA_NAME_GIVEN) \ + M(238, FORMAT_VERSION_TOO_OLD) \ + M(239, CANNOT_MUNMAP) \ + M(240, CANNOT_MREMAP) \ + M(241, MEMORY_LIMIT_EXCEEDED) \ + M(242, TABLE_IS_READ_ONLY) \ + M(243, NOT_ENOUGH_SPACE) \ + M(244, UNEXPECTED_ZOOKEEPER_ERROR) \ + M(246, CORRUPTED_DATA) \ + M(247, INCORRECT_MARK) \ + M(248, INVALID_PARTITION_VALUE) \ + M(250, NOT_ENOUGH_BLOCK_NUMBERS) \ + M(251, NO_SUCH_REPLICA) \ + M(252, TOO_MANY_PARTS) \ + M(253, REPLICA_IS_ALREADY_EXIST) \ + M(254, NO_ACTIVE_REPLICAS) \ + M(255, TOO_MANY_RETRIES_TO_FETCH_PARTS) \ + M(256, PARTITION_ALREADY_EXISTS) \ + M(257, PARTITION_DOESNT_EXIST) \ + M(258, UNION_ALL_RESULT_STRUCTURES_MISMATCH) \ + M(260, CLIENT_OUTPUT_FORMAT_SPECIFIED) \ + M(261, UNKNOWN_BLOCK_INFO_FIELD) \ + M(262, BAD_COLLATION) \ + M(263, CANNOT_COMPILE_CODE) \ + M(264, INCOMPATIBLE_TYPE_OF_JOIN) \ + M(265, NO_AVAILABLE_REPLICA) \ + M(266, MISMATCH_REPLICAS_DATA_SOURCES) \ + M(267, STORAGE_DOESNT_SUPPORT_PARALLEL_REPLICAS) \ + M(268, CPUID_ERROR) \ + M(269, INFINITE_LOOP) \ + M(270, CANNOT_COMPRESS) \ + M(271, CANNOT_DECOMPRESS) \ + M(272, CANNOT_IO_SUBMIT) \ + M(273, CANNOT_IO_GETEVENTS) \ + M(274, AIO_READ_ERROR) \ + M(275, AIO_WRITE_ERROR) \ + M(277, INDEX_NOT_USED) \ + M(279, ALL_CONNECTION_TRIES_FAILED) \ + M(280, NO_AVAILABLE_DATA) \ + M(281, DICTIONARY_IS_EMPTY) \ + M(282, INCORRECT_INDEX) \ + M(283, UNKNOWN_DISTRIBUTED_PRODUCT_MODE) \ + M(284, WRONG_GLOBAL_SUBQUERY) \ + M(285, TOO_FEW_LIVE_REPLICAS) \ + M(286, UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE) \ + M(287, UNKNOWN_FORMAT_VERSION) \ + M(288, DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED) \ + M(289, REPLICA_IS_NOT_IN_QUORUM) \ + M(290, LIMIT_EXCEEDED) \ + M(291, DATABASE_ACCESS_DENIED) \ + M(293, MONGODB_CANNOT_AUTHENTICATE) \ + M(294, INVALID_BLOCK_EXTRA_INFO) \ + M(295, RECEIVED_EMPTY_DATA) \ + M(296, NO_REMOTE_SHARD_FOUND) \ + M(297, SHARD_HAS_NO_CONNECTIONS) \ + M(298, CANNOT_PIPE) \ + M(299, CANNOT_FORK) \ + M(300, CANNOT_DLSYM) \ + M(301, CANNOT_CREATE_CHILD_PROCESS) \ + M(302, CHILD_WAS_NOT_EXITED_NORMALLY) \ + M(303, CANNOT_SELECT) \ + M(304, CANNOT_WAITPID) \ + M(305, TABLE_WAS_NOT_DROPPED) \ + M(306, TOO_DEEP_RECURSION) \ + M(307, TOO_MANY_BYTES) \ + M(308, UNEXPECTED_NODE_IN_ZOOKEEPER) \ + M(309, FUNCTION_CANNOT_HAVE_PARAMETERS) \ + M(317, INVALID_SHARD_WEIGHT) \ + M(318, INVALID_CONFIG_PARAMETER) \ + M(319, UNKNOWN_STATUS_OF_INSERT) \ + M(321, VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE) \ + M(335, BARRIER_TIMEOUT) \ + M(336, UNKNOWN_DATABASE_ENGINE) \ + M(337, DDL_GUARD_IS_ACTIVE) \ + M(341, UNFINISHED) \ + M(342, METADATA_MISMATCH) \ + M(344, SUPPORT_IS_DISABLED) \ + M(345, TABLE_DIFFERS_TOO_MUCH) \ + M(346, CANNOT_CONVERT_CHARSET) \ + M(347, CANNOT_LOAD_CONFIG) \ + M(349, CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN) \ + M(350, INCOMPATIBLE_SOURCE_TABLES) \ + M(351, AMBIGUOUS_TABLE_NAME) \ + M(352, AMBIGUOUS_COLUMN_NAME) \ + M(353, INDEX_OF_POSITIONAL_ARGUMENT_IS_OUT_OF_RANGE) \ + M(354, ZLIB_INFLATE_FAILED) \ + M(355, ZLIB_DEFLATE_FAILED) \ + M(356, BAD_LAMBDA) \ + M(357, RESERVED_IDENTIFIER_NAME) \ + M(358, INTO_OUTFILE_NOT_ALLOWED) \ + M(359, TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT) \ + M(360, CANNOT_CREATE_CHARSET_CONVERTER) \ + M(361, SEEK_POSITION_OUT_OF_BOUND) \ + M(362, CURRENT_WRITE_BUFFER_IS_EXHAUSTED) \ + M(363, CANNOT_CREATE_IO_BUFFER) \ + M(364, RECEIVED_ERROR_TOO_MANY_REQUESTS) \ + M(366, SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT) \ + M(367, TOO_MANY_FETCHES) \ + M(369, ALL_REPLICAS_ARE_STALE) \ + M(370, DATA_TYPE_CANNOT_BE_USED_IN_TABLES) \ + M(371, INCONSISTENT_CLUSTER_DEFINITION) \ + M(372, SESSION_NOT_FOUND) \ + M(373, SESSION_IS_LOCKED) \ + M(374, INVALID_SESSION_TIMEOUT) \ + M(375, CANNOT_DLOPEN) \ + M(376, CANNOT_PARSE_UUID) \ + M(377, ILLEGAL_SYNTAX_FOR_DATA_TYPE) \ + M(378, DATA_TYPE_CANNOT_HAVE_ARGUMENTS) \ + M(379, UNKNOWN_STATUS_OF_DISTRIBUTED_DDL_TASK) \ + M(380, CANNOT_KILL) \ + M(381, HTTP_LENGTH_REQUIRED) \ + M(382, CANNOT_LOAD_CATBOOST_MODEL) \ + M(383, CANNOT_APPLY_CATBOOST_MODEL) \ + M(384, PART_IS_TEMPORARILY_LOCKED) \ + M(385, MULTIPLE_STREAMS_REQUIRED) \ + M(386, NO_COMMON_TYPE) \ + M(387, DICTIONARY_ALREADY_EXISTS) \ + M(388, CANNOT_ASSIGN_OPTIMIZE) \ + M(389, INSERT_WAS_DEDUPLICATED) \ + M(390, CANNOT_GET_CREATE_TABLE_QUERY) \ + M(391, EXTERNAL_LIBRARY_ERROR) \ + M(392, QUERY_IS_PROHIBITED) \ + M(393, THERE_IS_NO_QUERY) \ + M(394, QUERY_WAS_CANCELLED) \ + M(395, FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) \ + M(396, TOO_MANY_ROWS_OR_BYTES) \ + M(397, QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW) \ + M(398, UNKNOWN_MUTATION_COMMAND) \ + M(399, FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT) \ + M(400, CANNOT_STAT) \ + M(401, FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME) \ + M(402, CANNOT_IOSETUP) \ + M(403, INVALID_JOIN_ON_EXPRESSION) \ + M(404, BAD_ODBC_CONNECTION_STRING) \ + M(405, PARTITION_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT) \ + M(406, TOP_AND_LIMIT_TOGETHER) \ + M(407, DECIMAL_OVERFLOW) \ + M(408, BAD_REQUEST_PARAMETER) \ + M(409, EXTERNAL_EXECUTABLE_NOT_FOUND) \ + M(410, EXTERNAL_SERVER_IS_NOT_RESPONDING) \ + M(411, PTHREAD_ERROR) \ + M(412, NETLINK_ERROR) \ + M(413, CANNOT_SET_SIGNAL_HANDLER) \ + M(415, ALL_REPLICAS_LOST) \ + M(416, REPLICA_STATUS_CHANGED) \ + M(417, EXPECTED_ALL_OR_ANY) \ + M(418, UNKNOWN_JOIN) \ + M(419, MULTIPLE_ASSIGNMENTS_TO_COLUMN) \ + M(420, CANNOT_UPDATE_COLUMN) \ + M(421, CANNOT_ADD_DIFFERENT_AGGREGATE_STATES) \ + M(422, UNSUPPORTED_URI_SCHEME) \ + M(423, CANNOT_GETTIMEOFDAY) \ + M(424, CANNOT_LINK) \ + M(425, SYSTEM_ERROR) \ + M(427, CANNOT_COMPILE_REGEXP) \ + M(428, UNKNOWN_LOG_LEVEL) \ + M(429, FAILED_TO_GETPWUID) \ + M(430, MISMATCHING_USERS_FOR_PROCESS_AND_DATA) \ + M(431, ILLEGAL_SYNTAX_FOR_CODEC_TYPE) \ + M(432, UNKNOWN_CODEC) \ + M(433, ILLEGAL_CODEC_PARAMETER) \ + M(434, CANNOT_PARSE_PROTOBUF_SCHEMA) \ + M(435, NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD) \ + M(436, PROTOBUF_BAD_CAST) \ + M(437, PROTOBUF_FIELD_NOT_REPEATED) \ + M(438, DATA_TYPE_CANNOT_BE_PROMOTED) \ + M(439, CANNOT_SCHEDULE_TASK) \ + M(440, INVALID_LIMIT_EXPRESSION) \ + M(441, CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING) \ + M(442, BAD_DATABASE_FOR_TEMPORARY_TABLE) \ + M(443, NO_COMMON_COLUMNS_WITH_PROTOBUF_SCHEMA) \ + M(444, UNKNOWN_PROTOBUF_FORMAT) \ + M(445, CANNOT_MPROTECT) \ + M(446, FUNCTION_NOT_ALLOWED) \ + M(447, HYPERSCAN_CANNOT_SCAN_TEXT) \ + M(448, BROTLI_READ_FAILED) \ + M(449, BROTLI_WRITE_FAILED) \ + M(450, BAD_TTL_EXPRESSION) \ + M(451, BAD_TTL_FILE) \ + M(452, SETTING_CONSTRAINT_VIOLATION) \ + M(453, MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES) \ + M(454, OPENSSL_ERROR) \ + M(455, SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY) \ + M(456, UNKNOWN_QUERY_PARAMETER) \ + M(457, BAD_QUERY_PARAMETER) \ + M(458, CANNOT_UNLINK) \ + M(459, CANNOT_SET_THREAD_PRIORITY) \ + M(460, CANNOT_CREATE_TIMER) \ + M(461, CANNOT_SET_TIMER_PERIOD) \ + M(462, CANNOT_DELETE_TIMER) \ + M(463, CANNOT_FCNTL) \ + M(464, CANNOT_PARSE_ELF) \ + M(465, CANNOT_PARSE_DWARF) \ + M(466, INSECURE_PATH) \ + M(467, CANNOT_PARSE_BOOL) \ + M(468, CANNOT_PTHREAD_ATTR) \ + M(469, VIOLATED_CONSTRAINT) \ + M(470, QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW) \ + M(471, INVALID_SETTING_VALUE) \ + M(472, READONLY_SETTING) \ + M(473, DEADLOCK_AVOIDED) \ + M(474, INVALID_TEMPLATE_FORMAT) \ + M(475, INVALID_WITH_FILL_EXPRESSION) \ + M(476, WITH_TIES_WITHOUT_ORDER_BY) \ + M(477, INVALID_USAGE_OF_INPUT) \ + M(478, UNKNOWN_POLICY) \ + M(479, UNKNOWN_DISK) \ + M(480, UNKNOWN_PROTOCOL) \ + M(481, PATH_ACCESS_DENIED) \ + M(482, DICTIONARY_ACCESS_DENIED) \ + M(483, TOO_MANY_REDIRECTS) \ + M(484, INTERNAL_REDIS_ERROR) \ + M(485, SCALAR_ALREADY_EXISTS) \ + M(487, CANNOT_GET_CREATE_DICTIONARY_QUERY) \ + M(488, UNKNOWN_DICTIONARY) \ + M(489, INCORRECT_DICTIONARY_DEFINITION) \ + M(490, CANNOT_FORMAT_DATETIME) \ + M(491, UNACCEPTABLE_URL) \ + M(492, ACCESS_ENTITY_NOT_FOUND) \ + M(493, ACCESS_ENTITY_ALREADY_EXISTS) \ + M(494, ACCESS_ENTITY_FOUND_DUPLICATES) \ + M(495, ACCESS_STORAGE_READONLY) \ + M(496, QUOTA_REQUIRES_CLIENT_KEY) \ + M(497, ACCESS_DENIED) \ + M(498, LIMIT_BY_WITH_TIES_IS_NOT_SUPPORTED) \ + M(499, S3_ERROR) \ + M(501, CANNOT_CREATE_DATABASE) \ + M(502, CANNOT_SIGQUEUE) \ + M(503, AGGREGATE_FUNCTION_THROW) \ + M(504, FILE_ALREADY_EXISTS) \ + M(505, CANNOT_DELETE_DIRECTORY) \ + M(506, UNEXPECTED_ERROR_CODE) \ + M(507, UNABLE_TO_SKIP_UNUSED_SHARDS) \ + M(508, UNKNOWN_ACCESS_TYPE) \ + M(509, INVALID_GRANT) \ + M(510, CACHE_DICTIONARY_UPDATE_FAIL) \ + M(511, UNKNOWN_ROLE) \ + M(512, SET_NON_GRANTED_ROLE) \ + M(513, UNKNOWN_PART_TYPE) \ + M(514, ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND) \ + M(515, INCORRECT_ACCESS_ENTITY_DEFINITION) \ + M(516, AUTHENTICATION_FAILED) \ + M(517, CANNOT_ASSIGN_ALTER) \ + M(518, CANNOT_COMMIT_OFFSET) \ + M(519, NO_REMOTE_SHARD_AVAILABLE) \ + M(520, CANNOT_DETACH_DICTIONARY_AS_TABLE) \ + M(521, ATOMIC_RENAME_FAIL) \ + M(523, UNKNOWN_ROW_POLICY) \ + M(524, ALTER_OF_COLUMN_IS_FORBIDDEN) \ + M(525, INCORRECT_DISK_INDEX) \ + M(526, UNKNOWN_VOLUME_TYPE) \ + M(527, NO_SUITABLE_FUNCTION_IMPLEMENTATION) \ + M(528, CASSANDRA_INTERNAL_ERROR) \ + M(529, NOT_A_LEADER) \ + M(530, CANNOT_CONNECT_RABBITMQ) \ + M(531, CANNOT_FSTAT) \ + M(532, LDAP_ERROR) \ + M(533, INCONSISTENT_RESERVATIONS) \ + M(534, NO_RESERVATIONS_PROVIDED) \ + M(535, UNKNOWN_RAID_TYPE) \ + M(536, CANNOT_RESTORE_FROM_FIELD_DUMP) \ + M(537, ILLEGAL_MYSQL_VARIABLE) \ + M(538, MYSQL_SYNTAX_ERROR) \ + M(539, CANNOT_BIND_RABBITMQ_EXCHANGE) \ + M(540, CANNOT_DECLARE_RABBITMQ_EXCHANGE) \ + M(541, CANNOT_CREATE_RABBITMQ_QUEUE_BINDING) \ + M(542, CANNOT_REMOVE_RABBITMQ_EXCHANGE) \ + M(543, UNKNOWN_MYSQL_DATATYPES_SUPPORT_LEVEL) \ + M(544, ROW_AND_ROWS_TOGETHER) \ + M(545, FIRST_AND_NEXT_TOGETHER) \ + M(546, NO_ROW_DELIMITER) \ + M(547, INVALID_RAID_TYPE) \ + M(548, UNKNOWN_VOLUME) \ \ - M(KEEPER_EXCEPTION, 999) \ - M(POCO_EXCEPTION, 1000) \ - M(STD_EXCEPTION, 1001) \ - M(UNKNOWN_EXCEPTION, 1002) \ + M(999, KEEPER_EXCEPTION) \ + M(1000, POCO_EXCEPTION) \ + M(1001, STD_EXCEPTION) \ + M(1002, UNKNOWN_EXCEPTION) \ \ - M(CONDITIONAL_TREE_PARENT_NOT_FOUND, 2001) \ - M(ILLEGAL_PROJECTION_MANIPULATOR, 2002) \ - M(UNRECOGNIZED_ARGUMENTS, 2003) + M(2001, CONDITIONAL_TREE_PARENT_NOT_FOUND) \ + M(2002, ILLEGAL_PROJECTION_MANIPULATOR) \ + M(2003, UNRECOGNIZED_ARGUMENTS) /* See END */ namespace DB @@ -531,7 +531,7 @@ namespace DB namespace ErrorCodes { - #define M(NAME, VALUE) extern const Value NAME = VALUE; + #define M(VALUE, NAME) extern const Value NAME = VALUE; APPLY_FOR_ERROR_CODES(M) #undef M @@ -543,7 +543,7 @@ namespace ErrorCodes std::string_view names[END+1]; ErrorCodesNames() { - #define M(NAME, VALUE) names[VALUE] = std::string_view(#NAME); + #define M(VALUE, NAME) names[VALUE] = std::string_view(#NAME); APPLY_FOR_ERROR_CODES(M) #undef M } From 2aa03627655ea4cd28a29e7690d9d6a2302cdeb0 Mon Sep 17 00:00:00 2001 From: Jacob Hayes Date: Sat, 31 Oct 2020 08:45:53 -0400 Subject: [PATCH 099/314] Add farmFingerprint64 function --- src/Functions/FunctionsHashing.cpp | 1 + src/Functions/FunctionsHashing.h | 13 +++++++++++++ 2 files changed, 14 insertions(+) diff --git a/src/Functions/FunctionsHashing.cpp b/src/Functions/FunctionsHashing.cpp index 900455a1015..5983e97a093 100644 --- a/src/Functions/FunctionsHashing.cpp +++ b/src/Functions/FunctionsHashing.cpp @@ -18,6 +18,7 @@ void registerFunctionsHashing(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 678214cdbad..17165e12e37 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -439,6 +439,18 @@ struct ImplCityHash64 static constexpr bool use_int_hash_for_pods = true; }; +// see farmhash.h for definition of NAMESPACE_FOR_HASH_FUNCTIONS +struct ImplFarmFingerprint64 +{ + static constexpr auto name = "farmFingerprint64"; + using ReturnType = UInt64; + using uint128_t = NAMESPACE_FOR_HASH_FUNCTIONS::uint128_t; + + static auto combineHashes(UInt64 h1, UInt64 h2) { return NAMESPACE_FOR_HASH_FUNCTIONS::Fingerprint(uint128_t(h1, h2)); } + static auto apply(const char * s, const size_t len) { return NAMESPACE_FOR_HASH_FUNCTIONS::Fingerprint64(s, len); } + static constexpr bool use_int_hash_for_pods = true; +}; + // see farmhash.h for definition of NAMESPACE_FOR_HASH_FUNCTIONS struct ImplFarmHash64 { @@ -1316,6 +1328,7 @@ using FunctionSHA256 = FunctionStringHashFixedString; #endif using FunctionSipHash128 = FunctionStringHashFixedString; using FunctionCityHash64 = FunctionAnyHash; +using FunctionFarmFingerprint64 = FunctionAnyHash; using FunctionFarmHash64 = FunctionAnyHash; using FunctionMetroHash64 = FunctionAnyHash; From c9ad4e8d96bce84382653756997b7324f50c281e Mon Sep 17 00:00:00 2001 From: Jacob Hayes Date: Sat, 31 Oct 2020 09:04:18 -0400 Subject: [PATCH 100/314] Add farmFingerprint64 docs --- docs/en/sql-reference/functions/hash-functions.md | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 73e9ef6254e..3594737c18a 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -153,15 +153,18 @@ A fast, decent-quality non-cryptographic hash function for a string obtained fro `URLHash(s, N)` – Calculates a hash from a string up to the N level in the URL hierarchy, without one of the trailing symbols `/`,`?` or `#` at the end, if present. Levels are the same as in URLHierarchy. This function is specific to Yandex.Metrica. +## farmFingerprint64 {#farmfingerprint64} + ## farmHash64 {#farmhash64} -Produces a 64-bit [FarmHash](https://github.com/google/farmhash) hash value. +Produces a 64-bit [FarmHash](https://github.com/google/farmhash) or Fingerprint value. Prefer `farmFingerprint64` for a stable and portable value. ``` sql +farmFingerprint64(par1, ...) farmHash64(par1, ...) ``` -The function uses the `Hash64` method from all [available methods](https://github.com/google/farmhash/blob/master/src/farmhash.h). +These functions use the `Fingerprint64` and `Hash64` method respectively from all [available methods](https://github.com/google/farmhash/blob/master/src/farmhash.h). **Parameters** From 2a747ce5af22fde28cfb811abf5a772f8d0a2781 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 1 Nov 2020 00:02:52 +0800 Subject: [PATCH 101/314] Allow WITH subqueries to take effect immediately --- src/Interpreters/ApplyWithSubqueryVisitor.cpp | 3 +-- .../01495_subqueries_in_with_statement.reference | 1 + .../0_stateless/01495_subqueries_in_with_statement.sql | 6 ++++++ 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ApplyWithSubqueryVisitor.cpp b/src/Interpreters/ApplyWithSubqueryVisitor.cpp index 805f425beac..d8ddbd2c2fa 100644 --- a/src/Interpreters/ApplyWithSubqueryVisitor.cpp +++ b/src/Interpreters/ApplyWithSubqueryVisitor.cpp @@ -16,10 +16,9 @@ void ApplyWithSubqueryVisitor::visit(ASTPtr & ast, const Data & data) std::optional new_data; if (auto with = node_select->with()) { - for (auto & child : with->children) - visit(child, data); for (auto & child : with->children) { + visit(child, new_data ? *new_data: data); if (auto * ast_with_elem = child->as()) { if (!new_data) diff --git a/tests/queries/0_stateless/01495_subqueries_in_with_statement.reference b/tests/queries/0_stateless/01495_subqueries_in_with_statement.reference index 16ca3450a74..c494604b4df 100644 --- a/tests/queries/0_stateless/01495_subqueries_in_with_statement.reference +++ b/tests/queries/0_stateless/01495_subqueries_in_with_statement.reference @@ -30,3 +30,4 @@ \N 42 42 +1764 diff --git a/tests/queries/0_stateless/01495_subqueries_in_with_statement.sql b/tests/queries/0_stateless/01495_subqueries_in_with_statement.sql index 8102ed29fa8..819346be129 100644 --- a/tests/queries/0_stateless/01495_subqueries_in_with_statement.sql +++ b/tests/queries/0_stateless/01495_subqueries_in_with_statement.sql @@ -70,4 +70,10 @@ SELECT max(n) m FROM test1 where test1.m=43 having max(n)=42; WITH test1 AS (SELECT n, null b, n+1 m FROM with_test where n = 42 order by n limit 4) SELECT max(n) m FROM test1 where b is null and test1.m=43 having m=42 limit 4; +with + test1 as (select n, null b, n+1 m from with_test where n = 42 order by n limit 4), + test2 as (select n + 1 as x, n - 1 as y from test1), + test3 as (select x * y as z from test2) +select z + 1 as q from test3; + drop table with_test ; From 73cfe76c49b49c7098abbeefff551e062772ff4f Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sat, 31 Oct 2020 20:16:49 +0300 Subject: [PATCH 102/314] Update order-by.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Удалил изменения. --- .../statements/select/order-by.md | 67 ------------------- 1 file changed, 67 deletions(-) diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index 314f41e6e5f..a4e5e3655c6 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -221,70 +221,3 @@ returns │ 1970-03-12 │ 1970-01-08 │ original │ └────────────┴────────────┴──────────┘ ``` - -## OFFSET FETCH Clause {#offset-fetch} - -`OFFSET` and `FETCH` allow you to retrieve just a portion of the rows that are generated by the rest of the query. - -``` sql -OFFSET offset_row_count {ROW | ROWS} FETCH {FIRST | NEXT} fetch_row_count {ROW | ROWS} {ONLY | WITH TIES} -``` - -The `FETCH` is an alternative to the [LIMIT] (../../../sql-reference/statements/select/limit.md) clause and retrieves rows from a query `SELECT` type. - -The `OFFSET` says to skip that many rows before beginning to return rows. - -For example, the following query - -``` sql -SELECT * FROM test_fetch ORDER BY a OFFSET 1 ROW FETCH FIRST 3 ROWS ONLY -``` - -is identical to the query - -``` sql -SELECT * FROM test_fetch ORDER BY a LIMIT 3 OFFSET 1 -``` - -When using `FETCH`, it is important to use an [ORDER BY] (../../../sql-reference/statements/select/order-by.md) clause that constrains the result rows into a unique order. Otherwise, you will get an unpredictable subset of the query's rows. - -`ROW` and `ROWS` as well as `FIRST` and `NEXT` are noise words that don't influence the effects of these conditions. - -The `ONLY` option is used to return rows that immediately follow the rows omitted by the `OFFSET`. - -The `WITH TIES` option is used to return any additional rows that tie for the last place in the result set according to the `ORDER BY` clause. The `ORDER BY` is mandatory in this case. - -!!! note "Note" - According to the standard, the `OFFSET` clause must come before the `FETCH` clause if both are present. - -### Example {#example} - -Input table: - -``` text -┌─a─┬─b─┐ -│ 1 │ 1 │ -│ 2 │ 1 │ -│ 3 │ 4 │ -│ 1 │ 3 │ -│ 5 │ 4 │ -│ 0 │ 6 │ -│ 5 │ 7 │ -└───┴───┘ -``` - -Query: - -``` sql -SELECT * FROM test_fetch ORDER BY a OFFSET 1 ROW FETCH FIRST 3 ROWS ONLY -``` - -Result: - -``` text -┌─a─┬─b─┐ -│ 1 │ 1 │ -│ 1 │ 3 │ -│ 2 │ 1 │ -└───┴───┘ -``` \ No newline at end of file From 5363c62fc5af620dc020ded9711818a20f5d89c2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 1 Nov 2020 08:56:30 +0300 Subject: [PATCH 103/314] Fix overflow check in ErrorCodes --- src/Common/ErrorCodes.h | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Common/ErrorCodes.h b/src/Common/ErrorCodes.h index 9afa4daae90..cc610c5d927 100644 --- a/src/Common/ErrorCodes.h +++ b/src/Common/ErrorCodes.h @@ -33,7 +33,12 @@ namespace ErrorCodes /// Add value for specified error_code. inline void increment(ErrorCode error_code) { - error_code = std::min(error_code, end()-1); + if (error_code >= end()) + { + /// For everything outside the range, use END. + /// (end() is the pointer pass the end, while END is the last value that has an element in values array). + error_code = end() - 1; + } values[error_code].fetch_add(1, std::memory_order_relaxed); } } From 08e523d5dea0bcb9a887ff8db400147b4a773870 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sun, 1 Nov 2020 17:58:37 +0300 Subject: [PATCH 104/314] Update settings.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Добавил описание настройки max_concurrent_queries_for_all_users. --- .../settings.md | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 368ef8e6f7c..dbece2d0cee 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -479,6 +479,24 @@ The maximum number of simultaneously processed requests. 100 ``` +## max_concurrent_queries_for_all_users {#max-concurrent-queries-for-all-users} + +The maximum number of simultaneously processed requests for all users. + +Default value: `0`. + +**Example** + +``` xml +99 +``` + +`max_concurrent_queries_for_all_users` can be set to 99 for all users and database administrator can set it to 100 for itself to run queries for investigation even when the server is overloaded. + +**See Also** + +- [max_concurrent_queries](#max-concurrent-queries) + ## max_connections {#max-connections} The maximum number of inbound connections. From 13fee1977877b2e262200c4cfbb28cb3c55fb5b1 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 2 Nov 2020 11:10:20 +0800 Subject: [PATCH 105/314] Allow parameterized functions in APPLY --- src/Parsers/ASTColumnsTransformers.cpp | 11 ++++++++--- src/Parsers/ASTColumnsTransformers.h | 3 +++ src/Parsers/ExpressionElementParsers.cpp | 19 ++++++++++++++++--- .../01470_columns_transformers.reference | 6 ++++++ .../01470_columns_transformers.sql | 4 ++++ 5 files changed, 37 insertions(+), 6 deletions(-) diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index fee606aec26..a8f39079902 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -30,16 +30,21 @@ void IASTColumnsTransformer::transform(const ASTPtr & transformer, ASTs & nodes) } } -void ASTColumnsApplyTransformer::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +void ASTColumnsApplyTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "APPLY" << (settings.hilite ? hilite_none : "") << "(" << func_name << ")"; + settings.ostr << (settings.hilite ? hilite_keyword : "") << "APPLY" << (settings.hilite ? hilite_none : "") << "(" << func_name; + if (parameters) + parameters->formatImpl(settings, state, frame); + settings.ostr << ")"; } void ASTColumnsApplyTransformer::transform(ASTs & nodes) const { for (auto & column : nodes) { - column = makeASTFunction(func_name, column); + auto function = makeASTFunction(func_name, column); + function->parameters = parameters; + column = function; } } diff --git a/src/Parsers/ASTColumnsTransformers.h b/src/Parsers/ASTColumnsTransformers.h index 4b7a933647e..38dbb80d7f8 100644 --- a/src/Parsers/ASTColumnsTransformers.h +++ b/src/Parsers/ASTColumnsTransformers.h @@ -18,10 +18,13 @@ public: ASTPtr clone() const override { auto res = std::make_shared(*this); + if (parameters) + res->parameters = parameters->clone(); return res; } void transform(ASTs & nodes) const override; String func_name; + ASTPtr parameters; protected: void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 3c45bd005a9..9abcf073e21 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1229,16 +1229,29 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e return false; ++pos; - String func_name; - if (!parseIdentifierOrStringLiteral(pos, expected, func_name)) + ASTPtr func_name; + if (!ParserIdentifier().parse(pos, func_name, expected)) return false; + ASTPtr expr_list_args; + if (pos->type == TokenType::OpeningRoundBracket) + { + ++pos; + if (!ParserExpressionList(false).parse(pos, expr_list_args, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + } + if (pos->type != TokenType::ClosingRoundBracket) return false; ++pos; auto res = std::make_shared(); - res->func_name = func_name; + res->func_name = getIdentifierName(func_name); + res->parameters = expr_list_args; node = std::move(res); return true; } diff --git a/tests/queries/0_stateless/01470_columns_transformers.reference b/tests/queries/0_stateless/01470_columns_transformers.reference index 2d8a1802289..a3b04de231a 100644 --- a/tests/queries/0_stateless/01470_columns_transformers.reference +++ b/tests/queries/0_stateless/01470_columns_transformers.reference @@ -77,3 +77,9 @@ SELECT toFloat64(k), j FROM columns_transformers +[110] [9] [173.5] +SELECT + quantiles(0.5)(i), + quantiles(0.5)(j), + quantiles(0.5)(k) +FROM columns_transformers diff --git a/tests/queries/0_stateless/01470_columns_transformers.sql b/tests/queries/0_stateless/01470_columns_transformers.sql index f95cee51fb0..f5178d139ac 100644 --- a/tests/queries/0_stateless/01470_columns_transformers.sql +++ b/tests/queries/0_stateless/01470_columns_transformers.sql @@ -41,4 +41,8 @@ EXPLAIN SYNTAX SELECT COLUMNS(i, j, k) APPLY(sum) from columns_transformers; SELECT i, j, COLUMNS(i, j, k) APPLY(toFloat64), COLUMNS(i, j) EXCEPT (i) from columns_transformers; EXPLAIN SYNTAX SELECT i, j, COLUMNS(i, j, k) APPLY(toFloat64), COLUMNS(i, j) EXCEPT (i) from columns_transformers; +-- APPLY with parameterized function +SELECT COLUMNS(i, j, k) APPLY(quantiles(0.5)) from columns_transformers; +EXPLAIN SYNTAX SELECT COLUMNS(i, j, k) APPLY(quantiles(0.5)) from columns_transformers; + DROP TABLE columns_transformers; From 99ee127620d9df5ad41b769571459c2941777b25 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 2 Nov 2020 10:50:38 +0300 Subject: [PATCH 106/314] Support `SETTINGS` clause for File engine Accept the usual user settings related to file formats. Most of the diff are the mechanistic code changes required to allow providing the required FormatSettings to the format factory. The File engine then extracts these settings from the `CREATE` query, and specifies them when creating the format parser. --- src/Formats/FormatFactory.cpp | 200 +++++++++--------- src/Formats/FormatFactory.h | 14 +- src/Formats/FormatSettings.h | 183 ++++++++-------- src/Processors/Formats/IRowInputFormat.cpp | 2 - src/Processors/Formats/IRowOutputFormat.h | 8 - .../Formats/Impl/ProtobufRowOutputFormat.cpp | 32 +-- .../Formats/Impl/ProtobufRowOutputFormat.h | 4 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 2 - src/Storages/Kafka/KafkaBlockOutputStream.cpp | 17 +- .../RabbitMQ/RabbitMQBlockOutputStream.cpp | 17 +- src/Storages/StorageFile.cpp | 89 +++++--- src/Storages/StorageFile.h | 8 +- src/TableFunctions/TableFunctionFile.cpp | 20 +- .../01544_file_engine_settings.reference | 2 + .../0_stateless/01544_file_engine_settings.sh | 23 ++ 15 files changed, 345 insertions(+), 276 deletions(-) create mode 100644 tests/queries/0_stateless/01544_file_engine_settings.reference create mode 100755 tests/queries/0_stateless/01544_file_engine_settings.sh diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 4dc5b816420..7011d2372b0 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -41,96 +41,75 @@ const FormatFactory::Creators & FormatFactory::getCreators(const String & name) } -static FormatSettings getInputFormatSetting(const Settings & settings, const Context & context) +FormatSettings getFormatSettings(const Context & context) { + const auto & settings = context.getSettingsRef(); + FormatSettings format_settings; - format_settings.csv.delimiter = settings.format_csv_delimiter; - format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes; + + format_settings.avro.allow_missing_fields = settings.input_format_avro_allow_missing_fields; + format_settings.avro.output_codec = settings.output_format_avro_codec; + format_settings.avro.output_sync_interval = settings.output_format_avro_sync_interval; + format_settings.avro.schema_registry_url = settings.format_avro_schema_registry_url.toString(); format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes; - format_settings.csv.unquoted_null_literal_as_null = settings.input_format_csv_unquoted_null_literal_as_null; + format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes; + format_settings.csv.crlf_end_of_line = settings.output_format_csv_crlf_end_of_line; + format_settings.csv.delimiter = settings.format_csv_delimiter; format_settings.csv.empty_as_default = settings.input_format_defaults_for_omitted_fields; format_settings.csv.input_format_enum_as_number = settings.input_format_csv_enum_as_number; - format_settings.null_as_default = settings.input_format_null_as_default; - format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions; - format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions; - format_settings.values.accurate_types_of_literals = settings.input_format_values_accurate_types_of_literals; - format_settings.with_names_use_header = settings.input_format_with_names_use_header; - format_settings.skip_unknown_fields = settings.input_format_skip_unknown_fields; - format_settings.import_nested_json = settings.input_format_import_nested_json; + format_settings.csv.unquoted_null_literal_as_null = settings.input_format_csv_unquoted_null_literal_as_null; + format_settings.custom.escaping_rule = settings.format_custom_escaping_rule; + format_settings.custom.field_delimiter = settings.format_custom_field_delimiter; + format_settings.custom.result_after_delimiter = settings.format_custom_result_after_delimiter; + format_settings.custom.result_after_delimiter = settings.format_custom_result_after_delimiter; + format_settings.custom.result_before_delimiter = settings.format_custom_result_before_delimiter; + format_settings.custom.row_after_delimiter = settings.format_custom_row_after_delimiter; + format_settings.custom.row_before_delimiter = settings.format_custom_row_before_delimiter; + format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter; format_settings.date_time_input_format = settings.date_time_input_format; + format_settings.date_time_output_format = settings.date_time_output_format; + format_settings.enable_streaming = settings.output_format_enable_streaming; + format_settings.import_nested_json = settings.input_format_import_nested_json; format_settings.input_allow_errors_num = settings.input_format_allow_errors_num; format_settings.input_allow_errors_ratio = settings.input_format_allow_errors_ratio; - format_settings.template_settings.resultset_format = settings.format_template_resultset; - format_settings.template_settings.row_format = settings.format_template_row; - format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter; - format_settings.tsv.empty_as_default = settings.input_format_tsv_empty_as_default; - format_settings.tsv.input_format_enum_as_number = settings.input_format_tsv_enum_as_number; + format_settings.json.escape_forward_slashes = settings.output_format_json_escape_forward_slashes; + format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers; + format_settings.json.quote_denormals = settings.output_format_json_quote_denormals; + format_settings.null_as_default = settings.input_format_null_as_default; + format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size; + format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8; + format_settings.pretty.color = settings.output_format_pretty_color; + format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width; + format_settings.pretty.max_rows = settings.output_format_pretty_max_rows; + format_settings.pretty.max_value_width = settings.output_format_pretty_max_value_width; + format_settings.pretty.output_format_pretty_row_numbers = settings.output_format_pretty_row_numbers; + format_settings.regexp.escaping_rule = settings.format_regexp_escaping_rule; + format_settings.regexp.regexp = settings.format_regexp; + format_settings.regexp.skip_unmatched = settings.format_regexp_skip_unmatched; format_settings.schema.format_schema = settings.format_schema; format_settings.schema.format_schema_path = context.getFormatSchemaPath(); format_settings.schema.is_server = context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER); - format_settings.custom.result_before_delimiter = settings.format_custom_result_before_delimiter; - format_settings.custom.result_after_delimiter = settings.format_custom_result_after_delimiter; - format_settings.custom.escaping_rule = settings.format_custom_escaping_rule; - format_settings.custom.field_delimiter = settings.format_custom_field_delimiter; - format_settings.custom.row_before_delimiter = settings.format_custom_row_before_delimiter; - format_settings.custom.row_after_delimiter = settings.format_custom_row_after_delimiter; - format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter; - format_settings.regexp.regexp = settings.format_regexp; - format_settings.regexp.escaping_rule = settings.format_regexp_escaping_rule; - format_settings.regexp.skip_unmatched = settings.format_regexp_skip_unmatched; + format_settings.skip_unknown_fields = settings.input_format_skip_unknown_fields; + format_settings.template_settings.resultset_format = settings.format_template_resultset; + format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter; + format_settings.template_settings.row_format = settings.format_template_row; + format_settings.tsv.crlf_end_of_line = settings.output_format_tsv_crlf_end_of_line; + format_settings.tsv.empty_as_default = settings.input_format_tsv_empty_as_default; + format_settings.tsv.input_format_enum_as_number = settings.input_format_tsv_enum_as_number; + format_settings.tsv.null_representation = settings.output_format_tsv_null_representation; + format_settings.values.accurate_types_of_literals = settings.input_format_values_accurate_types_of_literals; + format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions; + format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions; + format_settings.with_names_use_header = settings.input_format_with_names_use_header; + format_settings.write_statistics = settings.output_format_write_statistics; /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context - if (context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER)) + if (format_settings.schema.is_server) { const Poco::URI & avro_schema_registry_url = settings.format_avro_schema_registry_url; if (!avro_schema_registry_url.empty()) context.getRemoteHostFilter().checkURL(avro_schema_registry_url); } - format_settings.avro.schema_registry_url = settings.format_avro_schema_registry_url.toString(); - format_settings.avro.allow_missing_fields = settings.input_format_avro_allow_missing_fields; - - return format_settings; -} - -static FormatSettings getOutputFormatSetting(const Settings & settings, const Context & context) -{ - FormatSettings format_settings; - format_settings.enable_streaming = settings.output_format_enable_streaming; - format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers; - format_settings.json.quote_denormals = settings.output_format_json_quote_denormals; - format_settings.json.escape_forward_slashes = settings.output_format_json_escape_forward_slashes; - format_settings.csv.delimiter = settings.format_csv_delimiter; - format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes; - format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes; - format_settings.csv.crlf_end_of_line = settings.output_format_csv_crlf_end_of_line; - format_settings.pretty.max_rows = settings.output_format_pretty_max_rows; - format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width; - format_settings.pretty.max_value_width = settings.output_format_pretty_max_value_width; - format_settings.pretty.color = settings.output_format_pretty_color; - format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? - FormatSettings::Pretty::Charset::ASCII : - FormatSettings::Pretty::Charset::UTF8; - format_settings.pretty.output_format_pretty_row_numbers = settings.output_format_pretty_row_numbers; - format_settings.template_settings.resultset_format = settings.format_template_resultset; - format_settings.template_settings.row_format = settings.format_template_row; - format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter; - format_settings.tsv.crlf_end_of_line = settings.output_format_tsv_crlf_end_of_line; - format_settings.tsv.null_representation = settings.output_format_tsv_null_representation; - format_settings.write_statistics = settings.output_format_write_statistics; - format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size; - format_settings.schema.format_schema = settings.format_schema; - format_settings.schema.format_schema_path = context.getFormatSchemaPath(); - format_settings.schema.is_server = context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER); - format_settings.custom.result_before_delimiter = settings.format_custom_result_before_delimiter; - format_settings.custom.result_after_delimiter = settings.format_custom_result_after_delimiter; - format_settings.custom.escaping_rule = settings.format_custom_escaping_rule; - format_settings.custom.field_delimiter = settings.format_custom_field_delimiter; - format_settings.custom.row_before_delimiter = settings.format_custom_row_before_delimiter; - format_settings.custom.row_after_delimiter = settings.format_custom_row_after_delimiter; - format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter; - format_settings.avro.output_codec = settings.output_format_avro_codec; - format_settings.avro.output_sync_interval = settings.output_format_avro_sync_interval; - format_settings.date_time_output_format = settings.date_time_output_format; return format_settings; } @@ -142,7 +121,7 @@ BlockInputStreamPtr FormatFactory::getInput( const Block & sample, const Context & context, UInt64 max_block_size, - ReadCallback callback) const + std::optional format_settings) const { if (name == "Native") return std::make_shared(buf, sample, 0); @@ -153,10 +132,12 @@ BlockInputStreamPtr FormatFactory::getInput( if (!input_getter) throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); - const Settings & settings = context.getSettingsRef(); - FormatSettings format_settings = getInputFormatSetting(settings, context); + if (!format_settings) + { + format_settings = getFormatSettings(context); + } - return input_getter(buf, sample, max_block_size, callback ? callback : ReadCallback(), format_settings); + return input_getter(buf, sample, max_block_size, {}, *format_settings); } const Settings & settings = context.getSettingsRef(); @@ -182,17 +163,21 @@ BlockInputStreamPtr FormatFactory::getInput( if (!input_getter) throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); - FormatSettings format_settings = getInputFormatSetting(settings, context); + if (!format_settings) + { + format_settings = getFormatSettings(context); + } RowInputFormatParams row_input_format_params; row_input_format_params.max_block_size = max_block_size; - row_input_format_params.allow_errors_num = format_settings.input_allow_errors_num; - row_input_format_params.allow_errors_ratio = format_settings.input_allow_errors_ratio; - row_input_format_params.callback = std::move(callback); + row_input_format_params.allow_errors_num = format_settings->input_allow_errors_num; + row_input_format_params.allow_errors_ratio = format_settings->input_allow_errors_ratio; row_input_format_params.max_execution_time = settings.max_execution_time; row_input_format_params.timeout_overflow_mode = settings.timeout_overflow_mode; - auto input_creator_params = ParallelParsingBlockInputStream::InputCreatorParams{sample, row_input_format_params, format_settings}; + auto input_creator_params = + ParallelParsingBlockInputStream::InputCreatorParams{sample, + row_input_format_params, *format_settings}; ParallelParsingBlockInputStream::Params params{buf, input_getter, input_creator_params, file_segmentation_engine, static_cast(settings.max_threads), @@ -200,13 +185,15 @@ BlockInputStreamPtr FormatFactory::getInput( return std::make_shared(params); } - auto format = getInputFormat(name, buf, sample, context, max_block_size, std::move(callback)); + auto format = getInputFormat(name, buf, sample, context, max_block_size, + format_settings); return std::make_shared(std::move(format)); } -BlockOutputStreamPtr FormatFactory::getOutput( - const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback, const bool ignore_no_row_delimiter) const +BlockOutputStreamPtr FormatFactory::getOutput(const String & name, + WriteBuffer & buf, const Block & sample, const Context & context, + WriteCallback callback, std::optional format_settings) const { if (!getCreators(name).output_processor_creator) { @@ -214,18 +201,23 @@ BlockOutputStreamPtr FormatFactory::getOutput( if (!output_getter) throw Exception("Format " + name + " is not suitable for output", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT); - const Settings & settings = context.getSettingsRef(); - FormatSettings format_settings = getOutputFormatSetting(settings, context); + if (!format_settings) + { + format_settings = getFormatSettings(context); + } /** Materialization is needed, because formats can use the functions `IDataType`, * which only work with full columns. */ return std::make_shared( - output_getter(buf, sample, std::move(callback), format_settings), sample); + output_getter(buf, sample, std::move(callback), *format_settings), + sample); } - auto format = getOutputFormat(name, buf, sample, context, std::move(callback), ignore_no_row_delimiter); - return std::make_shared(std::make_shared(format), sample); + auto format = getOutputFormat(name, buf, sample, context, std::move(callback), + format_settings); + return std::make_shared( + std::make_shared(format), sample); } @@ -235,24 +227,27 @@ InputFormatPtr FormatFactory::getInputFormat( const Block & sample, const Context & context, UInt64 max_block_size, - ReadCallback callback) const + std::optional format_settings) const { const auto & input_getter = getCreators(name).input_processor_creator; if (!input_getter) throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); const Settings & settings = context.getSettingsRef(); - FormatSettings format_settings = getInputFormatSetting(settings, context); + + if (!format_settings) + { + format_settings = getFormatSettings(context); + } RowInputFormatParams params; params.max_block_size = max_block_size; - params.allow_errors_num = format_settings.input_allow_errors_num; - params.allow_errors_ratio = format_settings.input_allow_errors_ratio; - params.callback = std::move(callback); + params.allow_errors_num = format_settings->input_allow_errors_num; + params.allow_errors_ratio = format_settings->input_allow_errors_ratio; params.max_execution_time = settings.max_execution_time; params.timeout_overflow_mode = settings.timeout_overflow_mode; - auto format = input_getter(buf, sample, params, format_settings); + auto format = input_getter(buf, sample, params, *format_settings); /// It's a kludge. Because I cannot remove context from values format. if (auto * values = typeid_cast(format.get())) @@ -263,26 +258,29 @@ InputFormatPtr FormatFactory::getInputFormat( OutputFormatPtr FormatFactory::getOutputFormat( - const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback, const bool ignore_no_row_delimiter) const + const String & name, WriteBuffer & buf, const Block & sample, + const Context & context, WriteCallback callback, + std::optional format_settings) const { const auto & output_getter = getCreators(name).output_processor_creator; if (!output_getter) throw Exception("Format " + name + " is not suitable for output", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT); - const Settings & settings = context.getSettingsRef(); - FormatSettings format_settings = getOutputFormatSetting(settings, context); + if (!format_settings) + { + format_settings = getFormatSettings(context); + } RowOutputFormatParams params; - params.ignore_no_row_delimiter = ignore_no_row_delimiter; params.callback = std::move(callback); /** TODO: Materialization is needed, because formats can use the functions `IDataType`, * which only work with full columns. */ - auto format = output_getter(buf, sample, params, format_settings); + auto format = output_getter(buf, sample, params, *format_settings); /// Enable auto-flush for streaming mode. Currently it is needed by INSERT WATCH query. - if (format_settings.enable_streaming) + if (format_settings->enable_streaming) format->setAutoFlush(); /// It's a kludge. Because I cannot remove context from MySQL format. diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index d49414e3944..619acd10e0f 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -16,6 +17,7 @@ namespace DB class Block; class Context; struct FormatSettings; +struct Settings; class ReadBuffer; class WriteBuffer; @@ -32,6 +34,7 @@ struct RowOutputFormatParams; using InputFormatPtr = std::shared_ptr; using OutputFormatPtr = std::shared_ptr; +FormatSettings getFormatSettings(const Context & context); /** Allows to create an IBlockInputStream or IBlockOutputStream by the name of the format. * Note: format and compression are independent things. @@ -105,10 +108,11 @@ public: const Block & sample, const Context & context, UInt64 max_block_size, - ReadCallback callback = {}) const; + std::optional format_settings = std::nullopt) const; BlockOutputStreamPtr getOutput(const String & name, WriteBuffer & buf, - const Block & sample, const Context & context, WriteCallback callback = {}, const bool ignore_no_row_delimiter = false) const; + const Block & sample, const Context & context, WriteCallback callback = {}, + std::optional format_settings = std::nullopt) const; InputFormatPtr getInputFormat( const String & name, @@ -116,10 +120,12 @@ public: const Block & sample, const Context & context, UInt64 max_block_size, - ReadCallback callback = {}) const; + std::optional format_settings = std::nullopt) const; OutputFormatPtr getOutputFormat( - const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback = {}, const bool ignore_no_row_delimiter = false) const; + const String & name, WriteBuffer & buf, const Block & sample, + const Context & context, WriteCallback callback = {}, + std::optional format_settings = std::nullopt) const; /// Register format by its name. void registerInputFormat(const String & name, InputCreator input_creator); diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 8d7c3cdb49f..d2a596bca4d 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -17,76 +17,6 @@ struct FormatSettings /// Option means that each chunk of data need to be formatted independently. Also each chunk will be flushed at the end of processing. bool enable_streaming = false; - struct JSON - { - bool quote_64bit_integers = true; - bool quote_denormals = true; - bool escape_forward_slashes = true; - }; - - JSON json; - - struct CSV - { - char delimiter = ','; - bool allow_single_quotes = true; - bool allow_double_quotes = true; - bool unquoted_null_literal_as_null = false; - bool empty_as_default = false; - bool crlf_end_of_line = false; - bool input_format_enum_as_number = false; - }; - - CSV csv; - - struct Pretty - { - UInt64 max_rows = 10000; - UInt64 max_column_pad_width = 250; - UInt64 max_value_width = 10000; - bool color = true; - - bool output_format_pretty_row_numbers = false; - - enum class Charset - { - UTF8, - ASCII, - }; - - Charset charset = Charset::UTF8; - }; - - Pretty pretty; - - struct Values - { - bool interpret_expressions = true; - bool deduce_templates_of_expressions = true; - bool accurate_types_of_literals = true; - }; - - Values values; - - struct Template - { - String resultset_format; - String row_format; - String row_between_delimiter; - }; - - Template template_settings; - - struct TSV - { - bool empty_as_default = false; - bool crlf_end_of_line = false; - String null_representation = "\\N"; - bool input_format_enum_as_number = false; - }; - - TSV tsv; - bool skip_unknown_fields = false; bool with_names_use_header = false; bool write_statistics = true; @@ -113,24 +43,29 @@ struct FormatSettings UInt64 input_allow_errors_num = 0; Float32 input_allow_errors_ratio = 0; - struct Arrow + struct { UInt64 row_group_size = 1000000; } arrow; - struct Parquet + struct { - UInt64 row_group_size = 1000000; - } parquet; + String schema_registry_url; + String output_codec; + UInt64 output_sync_interval = 16 * 1024; + bool allow_missing_fields = false; + } avro; - struct Schema + struct CSV { - std::string format_schema; - std::string format_schema_path; - bool is_server = false; - }; - - Schema schema; + char delimiter = ','; + bool allow_single_quotes = true; + bool allow_double_quotes = true; + bool unquoted_null_literal_as_null = false; + bool empty_as_default = false; + bool crlf_end_of_line = false; + bool input_format_enum_as_number = false; + } csv; struct Custom { @@ -141,29 +76,89 @@ struct FormatSettings std::string row_between_delimiter; std::string field_delimiter; std::string escaping_rule; - }; + } custom; - Custom custom; - - struct Avro + struct { - String schema_registry_url; - String output_codec; - UInt64 output_sync_interval = 16 * 1024; - bool allow_missing_fields = false; - }; + bool quote_64bit_integers = true; + bool quote_denormals = true; + bool escape_forward_slashes = true; + bool serialize_as_strings = false; + } json; - Avro avro; + struct + { + UInt64 row_group_size = 1000000; + } parquet; - struct Regexp + struct Pretty + { + UInt64 max_rows = 10000; + UInt64 max_column_pad_width = 250; + UInt64 max_value_width = 10000; + bool color = true; + + bool output_format_pretty_row_numbers = false; + + enum class Charset + { + UTF8, + ASCII, + }; + + Charset charset = Charset::UTF8; + } pretty; + + struct + { + bool write_row_delimiters = true; + /** + * Some buffers (kafka / rabbit) split the rows internally using callback + * so we can push there formats without framing / delimiters (like + * ProtobufSingle). In other cases you can't write more than single row + * in unframed format. + * Not sure we need this parameter at all, it only serves as an additional + * safety check in ProtobufSingle format, but exporting constant-size + * records w/o delimiters might be generally useful, not only for Kafka. + */ + bool allow_many_rows_no_delimiters = false; + } protobuf; + + struct { std::string regexp; std::string escaping_rule; bool skip_unmatched = false; - }; + } regexp; - Regexp regexp; + struct + { + std::string format_schema; + std::string format_schema_path; + bool is_server = false; + } schema; + struct + { + String resultset_format; + String row_format; + String row_between_delimiter; + } template_settings; + + struct + { + bool empty_as_default = false; + bool crlf_end_of_line = false; + String null_representation = "\\N"; + bool input_format_enum_as_number = false; + } tsv; + + struct + { + bool interpret_expressions = true; + bool deduce_templates_of_expressions = true; + bool accurate_types_of_literals = true; + } values; }; } diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index 12d4db1f4a8..48cfdb12d8b 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -63,8 +63,6 @@ Chunk IRowInputFormat::generate() info.read_columns.clear(); if (!readRow(columns, info)) break; - if (params.callback) - params.callback(); for (size_t column_idx = 0; column_idx < info.read_columns.size(); ++column_idx) { diff --git a/src/Processors/Formats/IRowOutputFormat.h b/src/Processors/Formats/IRowOutputFormat.h index 4312691ea5e..4fb94f7b7f7 100644 --- a/src/Processors/Formats/IRowOutputFormat.h +++ b/src/Processors/Formats/IRowOutputFormat.h @@ -15,14 +15,6 @@ struct RowOutputFormatParams // Callback used to indicate that another row is written. WriteCallback callback; - - /** - * some buffers (kafka / rabbit) split the rows internally using callback - * so we can push there formats without framing / delimiters - * (like ProtobufSingle). In other cases you can't write more than single row - * in unframed format. - */ - bool ignore_no_row_delimiter = false; }; class WriteBuffer; diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp index 930a83c52da..3c885e80e31 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp @@ -23,18 +23,22 @@ ProtobufRowOutputFormat::ProtobufRowOutputFormat( const Block & header, const RowOutputFormatParams & params_, const FormatSchemaInfo & format_schema, - const bool use_length_delimiters_) + const FormatSettings & settings) : IRowOutputFormat(header, out_, params_) , data_types(header.getDataTypes()) - , writer(out, ProtobufSchemas::instance().getMessageTypeForFormatSchema(format_schema), header.getNames(), use_length_delimiters_) - , throw_on_multiple_rows_undelimited(!use_length_delimiters_ && !params_.ignore_no_row_delimiter) + , writer(out, + ProtobufSchemas::instance().getMessageTypeForFormatSchema(format_schema), + header.getNames(), settings.protobuf.write_row_delimiters) + , allow_only_one_row( + !settings.protobuf.write_row_delimiters + && !settings.protobuf.allow_many_rows_no_delimiters) { value_indices.resize(header.columns()); } void ProtobufRowOutputFormat::write(const Columns & columns, size_t row_num) { - if (throw_on_multiple_rows_undelimited && !first_row) + if (allow_only_one_row && !first_row) { throw Exception("The ProtobufSingle format can't be used to write multiple rows because this format doesn't have any row delimiter.", ErrorCodes::NO_ROW_DELIMITER); } @@ -51,19 +55,23 @@ void ProtobufRowOutputFormat::write(const Columns & columns, size_t row_num) void registerOutputFormatProcessorProtobuf(FormatFactory & factory) { - for (bool use_length_delimiters : {false, true}) + for (bool write_row_delimiters : {false, true}) { factory.registerOutputFormatProcessor( - use_length_delimiters ? "Protobuf" : "ProtobufSingle", - [use_length_delimiters](WriteBuffer & buf, + write_row_delimiters ? "Protobuf" : "ProtobufSingle", + [write_row_delimiters](WriteBuffer & buf, const Block & header, const RowOutputFormatParams & params, - const FormatSettings & settings) + const FormatSettings & _settings) { - return std::make_shared(buf, header, params, - FormatSchemaInfo(settings.schema.format_schema, "Protobuf", true, - settings.schema.is_server, settings.schema.format_schema_path), - use_length_delimiters); + FormatSettings settings = _settings; + settings.protobuf.write_row_delimiters = write_row_delimiters; + return std::make_shared( + buf, header, params, + FormatSchemaInfo(settings.schema.format_schema, "Protobuf", + true, settings.schema.is_server, + settings.schema.format_schema_path), + settings); }); } } diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h index 740efcfa24c..847f7607ff5 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h @@ -41,7 +41,7 @@ public: const Block & header, const RowOutputFormatParams & params_, const FormatSchemaInfo & format_schema, - const bool use_length_delimiters_); + const FormatSettings & settings); String getName() const override { return "ProtobufRowOutputFormat"; } @@ -53,7 +53,7 @@ private: DataTypes data_types; ProtobufWriter writer; std::vector value_indices; - const bool throw_on_multiple_rows_undelimited; + const bool allow_only_one_row; }; } diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index de5a1b71580..c3b753e7261 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -54,8 +54,6 @@ Chunk ValuesBlockInputFormat::generate() if (buf.eof() || *buf.position() == ';') break; readRow(columns, rows_in_block); - if (params.callback) - params.callback(); } catch (Exception & e) { diff --git a/src/Storages/Kafka/KafkaBlockOutputStream.cpp b/src/Storages/Kafka/KafkaBlockOutputStream.cpp index 9d7fe465d44..e7bf562339f 100644 --- a/src/Storages/Kafka/KafkaBlockOutputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockOutputStream.cpp @@ -32,13 +32,16 @@ void KafkaBlockOutputStream::writePrefix() if (!buffer) throw Exception("Failed to create Kafka producer!", ErrorCodes::CANNOT_CREATE_IO_BUFFER); - child = FormatFactory::instance().getOutput( - storage.getFormatName(), *buffer, getHeader(), *context, [this](const Columns & columns, size_t row) - { - buffer->countRow(columns, row); - }, - /* ignore_no_row_delimiter = */ true - ); + auto format_settings = getFormatSettings(*context); + format_settings.protobuf.allow_many_rows_no_delimiters = true; + + child = FormatFactory::instance().getOutput(storage.getFormatName(), *buffer, + getHeader(), *context, + [this](const Columns & columns, size_t row) + { + buffer->countRow(columns, row); + }, + format_settings); } void KafkaBlockOutputStream::write(const Block & block) diff --git a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp index 76129dee30d..b3bd57bdd0b 100644 --- a/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp +++ b/src/Storages/RabbitMQ/RabbitMQBlockOutputStream.cpp @@ -42,13 +42,16 @@ void RabbitMQBlockOutputStream::writePrefix() buffer->activateWriting(); - child = FormatFactory::instance().getOutput( - storage.getFormatName(), *buffer, getHeader(), context, [this](const Columns & /* columns */, size_t /* rows */) - { - buffer->countRow(); - }, - /* ignore_no_row_delimiter = */ true - ); + auto format_settings = getFormatSettings(context); + format_settings.protobuf.allow_many_rows_no_delimiters = true; + + child = FormatFactory::instance().getOutput(storage.getFormatName(), *buffer, + getHeader(), context, + [this](const Columns & /* columns */, size_t /* rows */) + { + buffer->countRow(); + }, + format_settings); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 8c7cd7b63d7..aa2696493f7 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include @@ -202,6 +203,7 @@ StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArgu StorageFile::StorageFile(CommonArguments args) : IStorage(args.table_id) , format_name(args.format_name) + , format_settings(args.format_settings) , compression_method(args.compression_method) , base_path(args.context.getPath()) { @@ -324,9 +326,11 @@ public: method = chooseCompressionMethod(current_path, storage->compression_method); } - read_buf = wrapReadBufferWithCompressionMethod(std::move(nested_buffer), method); - reader = FormatFactory::instance().getInput( - storage->format_name, *read_buf, metadata_snapshot->getSampleBlock(), context, max_block_size); + read_buf = wrapReadBufferWithCompressionMethod( + std::move(nested_buffer), method); + reader = FormatFactory::instance().getInput(storage->format_name, + *read_buf, metadata_snapshot->getSampleBlock(), context, + max_block_size, storage->format_settings); if (columns_description.hasDefaults()) reader = std::make_shared(reader, columns_description, context); @@ -430,8 +434,11 @@ Pipe StorageFile::read( pipes.reserve(num_streams); for (size_t i = 0; i < num_streams; ++i) + { pipes.emplace_back(std::make_shared( - this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns())); + this_ptr, metadata_snapshot, context, max_block_size, files_info, + metadata_snapshot->getColumns())); + } return Pipe::unitePipes(std::move(pipes)); } @@ -444,7 +451,8 @@ public: StorageFile & storage_, const StorageMetadataPtr & metadata_snapshot_, const CompressionMethod compression_method, - const Context & context) + const Context & context, + const FormatSettings & format_settings) : storage(storage_) , metadata_snapshot(metadata_snapshot_) , lock(storage.rwlock) @@ -472,7 +480,9 @@ public: write_buf = wrapWriteBufferWithCompressionMethod(std::move(naked_buffer), compression_method, 3); - writer = FormatFactory::instance().getOutput(storage.format_name, *write_buf, metadata_snapshot->getSampleBlock(), context); + writer = FormatFactory::instance().getOutput(storage.format_name, + *write_buf, metadata_snapshot->getSampleBlock(), context, + {}, format_settings); } Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } @@ -521,7 +531,8 @@ BlockOutputStreamPtr StorageFile::write( path = paths[0]; return std::make_shared(*this, metadata_snapshot, - chooseCompressionMethod(path, compression_method), context); + chooseCompressionMethod(path, compression_method), context, + format_settings); } Strings StorageFile::getDataPaths() const @@ -581,32 +592,54 @@ void StorageFile::truncate( void registerStorageFile(StorageFactory & factory) { + StorageFactory::StorageFeatures storage_features{ + .supports_settings = true, + .source_access_type = AccessType::FILE + }; + factory.registerStorage( "File", - [](const StorageFactory::Arguments & args) + [](const StorageFactory::Arguments & factory_args) { - ASTs & engine_args = args.engine_args; + StorageFile::CommonArguments storage_args{ + .table_id = factory_args.table_id, + .columns = factory_args.columns, + .constraints = factory_args.constraints, + .context = factory_args.context + }; - if (!(engine_args.size() >= 1 && engine_args.size() <= 3)) // NOLINT + ASTs & engine_args_ast = factory_args.engine_args; + + if (!(engine_args_ast.size() >= 1 && engine_args_ast.size() <= 3)) // NOLINT throw Exception( "Storage File requires from 1 to 3 arguments: name of used format, source and compression_method.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); - String format_name = engine_args[0]->as().value.safeGet(); + engine_args_ast[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args_ast[0], factory_args.local_context); + storage_args.format_name = engine_args_ast[0]->as().value.safeGet(); - String compression_method; - StorageFile::CommonArguments common_args{ - args.table_id, format_name, compression_method, args.columns, args.constraints, args.context}; + if (factory_args.storage_def->settings) + { + Context local_context_copy = factory_args.local_context; + local_context_copy.applySettingsChanges( + factory_args.storage_def->settings->changes); + storage_args.format_settings = getFormatSettings( + local_context_copy); + } + else + { + storage_args.format_settings = getFormatSettings( + factory_args.local_context); + } - if (engine_args.size() == 1) /// Table in database - return StorageFile::create(args.relative_data_path, common_args); + if (engine_args_ast.size() == 1) /// Table in database + return StorageFile::create(factory_args.relative_data_path, storage_args); /// Will use FD if engine_args[1] is int literal or identifier with std* name int source_fd = -1; String source_path; - if (auto opt_name = tryGetIdentifierName(engine_args[1])) + if (auto opt_name = tryGetIdentifierName(engine_args_ast[1])) { if (*opt_name == "stdin") source_fd = STDIN_FILENO; @@ -618,7 +651,7 @@ void registerStorageFile(StorageFactory & factory) throw Exception( "Unknown identifier '" + *opt_name + "' in second arg of File storage constructor", ErrorCodes::UNKNOWN_IDENTIFIER); } - else if (const auto * literal = engine_args[1]->as()) + else if (const auto * literal = engine_args_ast[1]->as()) { auto type = literal->value.getType(); if (type == Field::Types::Int64) @@ -631,23 +664,23 @@ void registerStorageFile(StorageFactory & factory) throw Exception("Second argument must be path or file descriptor", ErrorCodes::BAD_ARGUMENTS); } - if (engine_args.size() == 3) + if (engine_args_ast.size() == 3) { - engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.local_context); - compression_method = engine_args[2]->as().value.safeGet(); + engine_args_ast[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args_ast[2], factory_args.local_context); + storage_args.compression_method = engine_args_ast[2]->as().value.safeGet(); } else - compression_method = "auto"; + storage_args.compression_method = "auto"; if (0 <= source_fd) /// File descriptor - return StorageFile::create(source_fd, common_args); + return StorageFile::create(source_fd, storage_args); else /// User's file - return StorageFile::create(source_path, args.context.getUserFilesPath(), common_args); + return StorageFile::create(source_path, factory_args.context.getUserFilesPath(), storage_args); }, - { - .source_access_type = AccessType::FILE, - }); + storage_features); } + + NamesAndTypesList StorageFile::getVirtuals() const { return NamesAndTypesList{ diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index f331538b4c7..695cd0d3912 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -50,9 +50,10 @@ public: struct CommonArguments { - const StorageID & table_id; - const std::string & format_name; - const std::string & compression_method; + StorageID table_id; + std::string format_name; + FormatSettings format_settings; + std::string compression_method; const ColumnsDescription & columns; const ConstraintsDescription & constraints; const Context & context; @@ -79,6 +80,7 @@ private: explicit StorageFile(CommonArguments args); std::string format_name; + FormatSettings format_settings; int table_fd = -1; String compression_method; diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 39de6dce92c..f7d76309a7f 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -1,10 +1,12 @@ -#include -#include -#include -#include #include -#include + #include "registerTableFunctions.h" +#include +#include +#include +#include +#include +#include namespace DB { @@ -12,7 +14,13 @@ StoragePtr TableFunctionFile::getStorage( const String & source, const String & format_, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const std::string & compression_method_) const { - StorageFile::CommonArguments args{StorageID(getDatabaseName(), table_name), format_, compression_method_, columns, ConstraintsDescription{}, global_context}; + StorageFile::CommonArguments args{StorageID(getDatabaseName(), table_name), + format_, + getFormatSettings(global_context), + compression_method_, + columns, + ConstraintsDescription{}, + global_context}; return StorageFile::create(source, global_context.getUserFilesPath(), args); } diff --git a/tests/queries/0_stateless/01544_file_engine_settings.reference b/tests/queries/0_stateless/01544_file_engine_settings.reference new file mode 100644 index 00000000000..d2afb8fc688 --- /dev/null +++ b/tests/queries/0_stateless/01544_file_engine_settings.reference @@ -0,0 +1,2 @@ +1|1 +1 1 diff --git a/tests/queries/0_stateless/01544_file_engine_settings.sh b/tests/queries/0_stateless/01544_file_engine_settings.sh new file mode 100755 index 00000000000..b13ec0f3db3 --- /dev/null +++ b/tests/queries/0_stateless/01544_file_engine_settings.sh @@ -0,0 +1,23 @@ +#!/usr/bin/env bash +set -eu + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +the_file="$CLICKHOUSE_TMP/01544-t.csv" +rm -f -- "$the_file" + +# We are going to check that format settings work for File engine, +# by creating a table with a non-default delimiter, and reading from it. +${CLICKHOUSE_LOCAL} --query " + create table t(a int, b int) engine File(CSV, '$the_file') settings format_csv_delimiter = '|'; + insert into t select 1 a, 1 b; +" + +# See what's in the file +cat "$the_file" + +${CLICKHOUSE_LOCAL} --query " + create table t(a int, b int) engine File(CSV, '$the_file') settings format_csv_delimiter = '|'; + select * from t; +" From ceb83602b8fdf28fae45feac73e61cb0de53c383 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 2 Nov 2020 15:07:01 +0300 Subject: [PATCH 107/314] Refactor --- src/Interpreters/ActionsVisitor.cpp | 7 +++---- src/Interpreters/ActionsVisitor.h | 6 ++---- src/Interpreters/ExpressionAnalyzer.cpp | 10 +++++----- src/Interpreters/ExpressionAnalyzer.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- 5 files changed, 13 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 202cda2d467..3054f4781d0 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -385,7 +385,7 @@ ActionsMatcher::Data::Data( const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, const NamesAndTypesList & source_columns_, ActionsDAGPtr actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, - bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_, bool has_having_) + bool no_subqueries_, bool no_makeset_, bool only_consts_, bool create_source_for_in_) : context(context_) , set_size_limit(set_size_limit_) , subquery_depth(subquery_depth_) @@ -395,8 +395,7 @@ ActionsMatcher::Data::Data( , no_subqueries(no_subqueries_) , no_makeset(no_makeset_) , only_consts(only_consts_) - , no_storage_or_local(no_storage_or_local_) - , has_having(has_having_) + , create_source_for_in(create_source_for_in_) , visit_depth(0) , actions_stack(std::move(actions), context) , next_unique_suffix(actions_stack.getLastActions().getIndex().size() + 1) @@ -1054,7 +1053,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su * In case that we have HAVING with IN subquery, we have to force creating set for it. * Also it doesn't make sence if it is GLOBAL IN or ordinary IN. */ - if ((!subquery_for_set.source && data.no_storage_or_local) || data.has_having) + if (!subquery_for_set.source && data.create_source_for_in) { auto interpreter = interpretSubquery(right_in_operand, data.context, data.subquery_depth, {}); subquery_for_set.source = std::make_unique(); diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 10583efa5b0..c2dd9c9b033 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -117,8 +117,7 @@ public: bool no_subqueries; bool no_makeset; bool only_consts; - bool no_storage_or_local; - bool has_having; + bool create_source_for_in; size_t visit_depth; ScopeStack actions_stack; @@ -132,8 +131,7 @@ public: Data(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, const NamesAndTypesList & source_columns_, ActionsDAGPtr actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, - bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_, - bool has_having); + bool no_subqueries_, bool no_makeset_, bool only_consts_, bool create_source_for_in_); /// Does result of the calculation already exists in the block. bool hasColumn(const String & column_name) const; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 6f062548653..516c12b2be7 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -396,7 +396,7 @@ void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, LogAST log; ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth, sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets, - no_subqueries, false, only_consts, !isRemoteStorage(), false); + no_subqueries, false, only_consts, !isRemoteStorage()); ActionsVisitor(visitor_data, log.stream()).visit(ast); actions = visitor_data.getActions(); } @@ -407,17 +407,17 @@ void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, bool no_sub LogAST log; ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth, sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets, - no_subqueries, true, only_consts, !isRemoteStorage(), false); + no_subqueries, true, only_consts, !isRemoteStorage()); ActionsVisitor(visitor_data, log.stream()).visit(ast); actions = visitor_data.getActions(); } -void ExpressionAnalyzer::getRootActionsHasHaving(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts) +void ExpressionAnalyzer::getRootActionsForHaving(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts) { LogAST log; ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth, sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets, - no_subqueries, false, only_consts, !isRemoteStorage(), true); + no_subqueries, false, only_consts, true); ActionsVisitor(visitor_data, log.stream()).visit(ast); actions = visitor_data.getActions(); } @@ -836,7 +836,7 @@ bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); step.required_output.push_back(select_query->having()->getColumnName()); - getRootActions(select_query->having(), only_types, step.actions()); + getRootActionsForHaving(select_query->having(), only_types, step.actions()); return true; } diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 622c5204257..bd027e5a613 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -151,7 +151,7 @@ protected: */ void getRootActionsNoMakeSet(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts = false); - void getRootActionsHasHaving(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts = false); + void getRootActionsForHaving(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts = false); /** Add aggregation keys to aggregation_keys, aggregate functions to aggregate_descriptions, * Create a set of columns aggregated_columns resulting after the aggregation, if any, diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 07c1942c08d..814350d2267 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -958,7 +958,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu preliminary_sort(); // If there is no global subqueries, we can run subqueries only when receive them on server. - if (expressions.hasHaving() || (!query_analyzer->hasGlobalSubqueries() && !subqueries_for_sets.empty())) + if (!query_analyzer->hasGlobalSubqueries() && !subqueries_for_sets.empty()) executeSubqueriesInSetsAndJoins(query_plan, subqueries_for_sets); } @@ -1071,7 +1071,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu } } - if (expressions.hasHaving() || (query_analyzer->hasGlobalSubqueries() && !subqueries_for_sets.empty())) + if (!subqueries_for_sets.empty() && (expressions.hasHaving() || query_analyzer->hasGlobalSubqueries())) executeSubqueriesInSetsAndJoins(query_plan, subqueries_for_sets); } From 4cfae808faa8b7158a5f7cae7ee124eefd16102a Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 2 Nov 2020 18:03:52 +0800 Subject: [PATCH 108/314] Query parameter type : Identifier --- .../ReplaceQueryParameterVisitor.cpp | 27 ++++++++ .../ReplaceQueryParameterVisitor.h | 2 + src/Parsers/ASTIdentifier.cpp | 62 +++++++++++++----- src/Parsers/ASTIdentifier.h | 9 ++- src/Parsers/ExpressionElementParsers.cpp | 65 ++++++++++++++++--- src/Parsers/ExpressionElementParsers.h | 22 ++++++- src/Parsers/ParserTablesInSelectQuery.cpp | 2 +- ...1550_query_identifier_parameters.reference | 5 ++ .../01550_query_identifier_parameters.sh | 10 +++ 9 files changed, 176 insertions(+), 28 deletions(-) create mode 100644 tests/queries/0_stateless/01550_query_identifier_parameters.reference create mode 100755 tests/queries/0_stateless/01550_query_identifier_parameters.sh diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index 3dabfb06770..9b4223b8947 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -5,8 +5,10 @@ #include #include #include +#include #include #include +#include #include #include @@ -25,6 +27,8 @@ void ReplaceQueryParameterVisitor::visit(ASTPtr & ast) { if (ast->as()) visitQueryParameter(ast); + else if (ast->as()) + visitIdentifier(ast); else visitChildren(ast); } @@ -71,4 +75,27 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) ast->setAlias(alias); } +void ReplaceQueryParameterVisitor::visitIdentifier(ASTPtr & ast) +{ + auto & ast_identifier = ast->as(); + if (ast_identifier.children.empty()) + return; + + auto & name_parts = ast_identifier.name_parts; + for (size_t i = 0, j = 0, size = name_parts.size(); i < size; ++i) + { + if (name_parts[i].empty()) + { + const auto & ast_param = ast_identifier.children[j++]->as(); + name_parts[i] = getParamValue(ast_param.name); + } + } + + if (!ast_identifier.semantic->special && name_parts.size() >= 2) + ast_identifier.semantic->table = ast_identifier.name_parts.end()[-2]; + + ast_identifier.resetFullName(); + ast_identifier.children.clear(); +} + } diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.h b/src/Interpreters/ReplaceQueryParameterVisitor.h index 3a84cd22acd..23e36df3fee 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.h +++ b/src/Interpreters/ReplaceQueryParameterVisitor.h @@ -9,6 +9,7 @@ namespace DB class ASTQueryParameter; /// Visit substitutions in a query, replace ASTQueryParameter with ASTLiteral. +/// Rebuild ASTIdentifiers if some parts are ASTQueryParameter. class ReplaceQueryParameterVisitor { public: @@ -21,6 +22,7 @@ public: private: const NameToNameMap & query_parameters; const String & getParamValue(const String & name); + void visitIdentifier(ASTPtr & ast); void visitQueryParameter(ASTPtr & ast); void visitChildren(ASTPtr & ast); }; diff --git a/src/Parsers/ASTIdentifier.cpp b/src/Parsers/ASTIdentifier.cpp index d980300a22a..5a66bc7891d 100644 --- a/src/Parsers/ASTIdentifier.cpp +++ b/src/Parsers/ASTIdentifier.cpp @@ -16,26 +16,48 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; } -ASTIdentifier::ASTIdentifier(const String & short_name) +ASTIdentifier::ASTIdentifier(const String & short_name, ASTPtr && name_param) : full_name(short_name), name_parts{short_name}, semantic(std::make_shared()) { - assert(!full_name.empty()); + if (name_param == nullptr) + assert(!full_name.empty()); + else + children.push_back(std::move(name_param)); } -ASTIdentifier::ASTIdentifier(std::vector && name_parts_, bool special) +ASTIdentifier::ASTIdentifier(std::vector && name_parts_, bool special, std::vector && name_params) : name_parts(name_parts_), semantic(std::make_shared()) { assert(!name_parts.empty()); - for (const auto & part [[maybe_unused]] : name_parts) - assert(!part.empty()); - semantic->special = special; semantic->legacy_compound = true; + if (!name_params.empty()) + { + size_t params = 0; + for (const auto & part [[maybe_unused]] : name_parts) + { + if (part.empty()) + ++params; + } + assert(params == name_params.size()); + children = std::move(name_params); + } + else + { + for (const auto & part [[maybe_unused]] : name_parts) + assert(!part.empty()); - if (!special && name_parts.size() >= 2) - semantic->table = name_parts.end()[-2]; + if (!special && name_parts.size() >= 2) + semantic->table = name_parts.end()[-2]; - resetFullName(); + resetFullName(); + } +} + +ASTPtr ASTIdentifier::getParam() const +{ + assert(full_name.empty() && children.size() == 1); + return children.front()->clone(); } ASTPtr ASTIdentifier::clone() const @@ -64,13 +86,16 @@ void ASTIdentifier::setShortName(const String & new_name) const String & ASTIdentifier::name() const { - assert(!name_parts.empty()); - assert(!full_name.empty()); + if (children.empty()) + { + assert(!name_parts.empty()); + assert(!full_name.empty()); + } return full_name; } -void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const +void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { auto format_element = [&](const String & elem_name) { @@ -82,17 +107,24 @@ void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, Form /// It could be compound but short if (!isShort()) { - for (size_t i = 0, size = name_parts.size(); i < size; ++i) + for (size_t i = 0, j = 0, size = name_parts.size(); i < size; ++i) { if (i != 0) settings.ostr << '.'; - format_element(name_parts[i]); + if (name_parts[i].empty()) + children[j++]->formatImpl(settings, state, frame); + else + format_element(name_parts[i]); } } else { - format_element(shortName()); + const auto & name = shortName(); + if (name.empty()) + children.front()->formatImpl(settings, state, frame); + else + format_element(name); } } diff --git a/src/Parsers/ASTIdentifier.h b/src/Parsers/ASTIdentifier.h index 59f698eab1c..205b3bb9ad1 100644 --- a/src/Parsers/ASTIdentifier.h +++ b/src/Parsers/ASTIdentifier.h @@ -2,6 +2,7 @@ #include +#include #include #include @@ -17,15 +18,19 @@ struct StorageID; /// Identifier (column, table or alias) class ASTIdentifier : public ASTWithAlias { + friend class ReplaceQueryParameterVisitor; public: UUID uuid = UUIDHelpers::Nil; - explicit ASTIdentifier(const String & short_name); - explicit ASTIdentifier(std::vector && name_parts, bool special = false); + explicit ASTIdentifier(const String & short_name, ASTPtr && name_param = {}); + explicit ASTIdentifier(std::vector && name_parts, bool special = false, std::vector && name_params = {}); /** Get the text that identifies this element. */ String getID(char delim) const override { return "Identifier" + (delim + name()); } + /** Get the query param out of a non-compound identifier. */ + ASTPtr getParam() const; + ASTPtr clone() const override; void collectIdentifierNames(IdentifierNameSet & set) const override { set.insert(name()); } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 3c45bd005a9..1761b0b4358 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -146,7 +146,7 @@ bool ParserSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } -bool ParserIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected &) +bool ParserIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { /// Identifier in backquotes or in double quotes if (pos->type == TokenType::QuotedIdentifier) @@ -172,7 +172,51 @@ bool ParserIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected &) ++pos; return true; } + else if (allow_query_parameter && pos->type == TokenType::OpeningCurlyBrace) + { + ++pos; + if (pos->type != TokenType::BareWord) + { + expected.add(pos, "substitution name (identifier)"); + return false; + } + String name(pos->begin, pos->end); + ++pos; + + if (pos->type != TokenType::Colon) + { + expected.add(pos, "colon between name and type"); + return false; + } + + ++pos; + + if (pos->type != TokenType::BareWord) + { + expected.add(pos, "substitution type (identifier)"); + return false; + } + + String type(pos->begin, pos->end); + ++pos; + + if (type != "Identifier") + { + expected.add(pos, "substitution type (identifier)"); + return false; + } + + if (pos->type != TokenType::ClosingCurlyBrace) + { + expected.add(pos, "closing curly brace"); + return false; + } + ++pos; + + node = std::make_shared("", std::make_shared(name, type)); + return true; + } return false; } @@ -180,14 +224,19 @@ bool ParserIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected &) bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr id_list; - if (!ParserList(std::make_unique(), std::make_unique(TokenType::Dot), false) - .parse(pos, id_list, expected)) + if (!ParserList(std::make_unique(allow_query_parameter), std::make_unique(TokenType::Dot), false) + .parse(pos, id_list, expected)) return false; std::vector parts; + std::vector params; const auto & list = id_list->as(); for (const auto & child : list.children) + { parts.emplace_back(getIdentifierName(child)); + if (parts.back() == "") + params.push_back(child->as()->getParam()); + } ParserKeyword s_uuid("UUID"); UUID uuid = UUIDHelpers::Nil; @@ -201,7 +250,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex uuid = parseFromString(ast_uuid->as()->value.get()); } - node = std::make_shared(std::move(parts)); + node = std::make_shared(std::move(parts), false, std::move(params)); node->as()->uuid = uuid; return true; @@ -1174,7 +1223,7 @@ bool ParserAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) bool ParserColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword columns("COLUMNS"); - ParserList columns_p(std::make_unique(), std::make_unique(TokenType::Comma), false); + ParserList columns_p(std::make_unique(true), std::make_unique(TokenType::Comma), false); ParserStringLiteral regex; if (!columns.ignore(pos, expected)) @@ -1252,7 +1301,7 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e auto parse_id = [&identifiers, &pos, &expected] { ASTPtr identifier; - if (!ParserIdentifier().parse(pos, identifier, expected)) + if (!ParserIdentifier(true).parse(pos, identifier, expected)) return false; identifiers.emplace_back(std::move(identifier)); @@ -1338,7 +1387,7 @@ bool ParserAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) bool ParserQualifiedAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - if (!ParserCompoundIdentifier().parse(pos, node, expected)) + if (!ParserCompoundIdentifier(false, true).parse(pos, node, expected)) return false; if (pos->type != TokenType::Dot) @@ -1475,7 +1524,7 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp || ParserFunction().parse(pos, node, expected) || ParserQualifiedAsterisk().parse(pos, node, expected) || ParserAsterisk().parse(pos, node, expected) - || ParserCompoundIdentifier().parse(pos, node, expected) + || ParserCompoundIdentifier(false, true).parse(pos, node, expected) || ParserSubstitution().parse(pos, node, expected) || ParserMySQLGlobalVariable().parse(pos, node, expected); } diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index 702d757761a..86cc3db538c 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -42,9 +42,12 @@ protected: */ class ParserIdentifier : public IParserBase { +public: + ParserIdentifier(bool allow_query_parameter_ = false) : allow_query_parameter(allow_query_parameter_) {} protected: const char * getName() const override { return "identifier"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + bool allow_query_parameter; }; @@ -54,12 +57,16 @@ protected: class ParserCompoundIdentifier : public IParserBase { public: - ParserCompoundIdentifier(bool table_name_with_optional_uuid_ = false) - : table_name_with_optional_uuid(table_name_with_optional_uuid_) {} + ParserCompoundIdentifier(bool table_name_with_optional_uuid_ = false, bool allow_query_parameter_ = false) + : table_name_with_optional_uuid(table_name_with_optional_uuid_), allow_query_parameter(allow_query_parameter_) + { + } + protected: const char * getName() const override { return "compound identifier"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; bool table_name_with_optional_uuid; + bool allow_query_parameter; }; /// Just * @@ -299,6 +306,17 @@ private: }; +/** Prepared statements. + * Parse query with parameter expression {name:type}. + */ +class ParserIdentifierOrSubstitution : public IParserBase +{ +protected: + const char * getName() const override { return "identifier substitution"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + + /** Prepared statements. * Parse query with parameter expression {name:type}. */ diff --git a/src/Parsers/ParserTablesInSelectQuery.cpp b/src/Parsers/ParserTablesInSelectQuery.cpp index a13baf69420..1264acefe64 100644 --- a/src/Parsers/ParserTablesInSelectQuery.cpp +++ b/src/Parsers/ParserTablesInSelectQuery.cpp @@ -23,7 +23,7 @@ bool ParserTableExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (!ParserWithOptionalAlias(std::make_unique(), true).parse(pos, res->subquery, expected) && !ParserWithOptionalAlias(std::make_unique(), true).parse(pos, res->table_function, expected) - && !ParserWithOptionalAlias(std::make_unique(), true).parse(pos, res->database_and_table_name, expected)) + && !ParserWithOptionalAlias(std::make_unique(false, true), true).parse(pos, res->database_and_table_name, expected)) return false; /// FINAL diff --git a/tests/queries/0_stateless/01550_query_identifier_parameters.reference b/tests/queries/0_stateless/01550_query_identifier_parameters.reference new file mode 100644 index 00000000000..751ee1ae00e --- /dev/null +++ b/tests/queries/0_stateless/01550_query_identifier_parameters.reference @@ -0,0 +1,5 @@ +0 +0 +0 +0 +45 diff --git a/tests/queries/0_stateless/01550_query_identifier_parameters.sh b/tests/queries/0_stateless/01550_query_identifier_parameters.sh new file mode 100755 index 00000000000..85ca67e4e3c --- /dev/null +++ b/tests/queries/0_stateless/01550_query_identifier_parameters.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --param_tbl 'numbers' --query 'select * from system.{tbl:Identifier} limit 1' +$CLICKHOUSE_CLIENT --param_db 'system' --param_tbl 'numbers' --query 'select * from {db:Identifier}.{tbl:Identifier} limit 1' +$CLICKHOUSE_CLIENT --param_col 'number' --query 'select {col:Identifier} from system.numbers limit 1' +$CLICKHOUSE_CLIENT --param_col 'number' --query 'select a.{col:Identifier} from system.numbers a limit 1' +$CLICKHOUSE_CLIENT --param_tbl 'numbers' --param_col 'number' --query 'select sum({tbl:Identifier}.{col:Identifier}) FROM (select * from system.{tbl:Identifier} limit 10) numbers' From 5633dfb402eb1a0da62656efc9946d273a4dec30 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 2 Nov 2020 16:15:40 +0300 Subject: [PATCH 109/314] Update ExpressionElementParsers.h --- src/Parsers/ExpressionElementParsers.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index 86cc3db538c..09c43a1f769 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -39,6 +39,7 @@ protected: /** An identifier, for example, x_yz123 or `something special` + * If allow_query_parameter_ = true, also parses substitutions in form {name:Identifier} */ class ParserIdentifier : public IParserBase { @@ -312,7 +313,7 @@ private: class ParserIdentifierOrSubstitution : public IParserBase { protected: - const char * getName() const override { return "identifier substitution"; } + const char * getName() const override { return "identifier or substitution"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; From db1e9750b935221b9ece4743a8e381a1215872c3 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 2 Nov 2020 17:35:06 +0300 Subject: [PATCH 110/314] fix build --- src/Formats/tests/tab_separated_streams.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Formats/tests/tab_separated_streams.cpp b/src/Formats/tests/tab_separated_streams.cpp index bb38d51cf98..bd733e4b9aa 100644 --- a/src/Formats/tests/tab_separated_streams.cpp +++ b/src/Formats/tests/tab_separated_streams.cpp @@ -38,8 +38,8 @@ try FormatSettings format_settings; - RowInputFormatParams in_params{DEFAULT_INSERT_BLOCK_SIZE, 0, 0, []{}}; - RowOutputFormatParams out_params{[](const Columns & /* columns */, size_t /* row */){},false}; + RowInputFormatParams in_params{DEFAULT_INSERT_BLOCK_SIZE, 0, 0}; + RowOutputFormatParams out_params{[](const Columns & /* columns */, size_t /* row */){}}; InputFormatPtr input_format = std::make_shared(sample, in_buf, in_params, false, false, format_settings); BlockInputStreamPtr block_input = std::make_shared(std::move(input_format)); From 5e512568977143f2a5172363a1b85d54d615eb4c Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 2 Nov 2020 23:04:18 +0800 Subject: [PATCH 111/314] Fix warning --- src/Parsers/ExpressionElementParsers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 1761b0b4358..d69e421e6cf 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -234,7 +234,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex for (const auto & child : list.children) { parts.emplace_back(getIdentifierName(child)); - if (parts.back() == "") + if (parts.back().empty()) params.push_back(child->as()->getParam()); } From 0f22a9dfcdf183fd81f762e0b0bf844b32d94b29 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 2 Nov 2020 19:13:42 +0300 Subject: [PATCH 112/314] fix build --- src/Processors/Formats/IRowInputFormat.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Processors/Formats/IRowInputFormat.h b/src/Processors/Formats/IRowInputFormat.h index 1931fba2a0d..14109f9c2be 100644 --- a/src/Processors/Formats/IRowInputFormat.h +++ b/src/Processors/Formats/IRowInputFormat.h @@ -27,9 +27,6 @@ struct RowInputFormatParams UInt64 allow_errors_num; Float64 allow_errors_ratio; - using ReadCallback = std::function; - ReadCallback callback; - Poco::Timespan max_execution_time = 0; OverflowMode timeout_overflow_mode = OverflowMode::THROW; }; From 364607d87d5e2dd417466f26315e6059ae118014 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 2 Nov 2020 19:18:18 +0300 Subject: [PATCH 113/314] Merge with master --- src/Storages/StorageProxy.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index 7b010476b22..bb5223d0e3d 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -102,12 +102,11 @@ public: } Pipe alterPartition( - const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) override { - return getNested()->alterPartition(query, metadata_snapshot, commands, context); + return getNested()->alterPartition(metadata_snapshot, commands, context); } void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override From dac670599521b5d86510b7109430d8ed50eb9f6b Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 2 Nov 2020 20:30:53 +0300 Subject: [PATCH 114/314] DROP PART and clear blocks in a single zookeeper op --- src/Storages/MergeTree/MergeTreeData.cpp | 8 +++ src/Storages/MergeTree/MergeTreeData.h | 2 + src/Storages/StorageMergeTree.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 54 ++++++++++----------- src/Storages/StorageReplicatedMergeTree.h | 1 + 5 files changed, 38 insertions(+), 31 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index cd776a661ed..fb7a7c02078 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2642,6 +2642,14 @@ void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition) global_context.checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, partition_size); } +void MergeTreeData::checkPartCanBeDropped(const ASTPtr & part_ast) +{ + String part_name = part_ast->as().value.safeGet(); + auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Committed}); + if (!part) + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in commited state", part_name); +} + void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String & name, bool moving_part, const Context & context) { String partition_id; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 889e4fb16b4..b5506d87a8b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -560,6 +560,8 @@ public: void checkPartitionCanBeDropped(const ASTPtr & partition) override; + void checkPartCanBeDropped(const ASTPtr & part); + size_t getColumnCompressedSize(const std::string & name) const { auto lock = lockParts(); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 0c5fb5d9e8c..844884714ab 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1117,9 +1117,7 @@ Pipe StorageMergeTree::alterPartition( { case PartitionCommand::DROP_PARTITION: if (command.part) - { - /// TODO(nv) what would be a good check here? - } + checkPartCanBeDropped(command.partition); else checkPartitionCanBeDropped(command.partition); dropPartition(command.partition, command.detach, command.part, query_context); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1cbb2fe800f..6dae5a2d7d5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4191,9 +4191,7 @@ Pipe StorageReplicatedMergeTree::alterPartition( { case PartitionCommand::DROP_PARTITION: if (command.part) - { - /// TODO(nv) what to check here? it is possible to drop a big partition by dropping small parts... - } + checkPartCanBeDropped(command.partition); else checkPartitionCanBeDropped(command.partition); dropPartition(command.partition, command.detach, command.part, query_context); @@ -5479,8 +5477,8 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper( } -void StorageReplicatedMergeTree::clearBlocksInPartition( - zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num) +void StorageReplicatedMergeTree::getClearBlocksInPartitionOps( + Coordination::Requests & ops, zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num) { Strings blocks; if (Coordination::Error::ZOK != zookeeper.tryGetChildren(zookeeper_path + "/blocks", blocks)) @@ -5497,7 +5495,6 @@ void StorageReplicatedMergeTree::clearBlocksInPartition( } } - zkutil::AsyncResponses to_delete_futures; for (auto & pair : get_futures) { const String & path = pair.first; @@ -5510,23 +5507,25 @@ void StorageReplicatedMergeTree::clearBlocksInPartition( Int64 block_num = 0; bool parsed = tryReadIntText(block_num, buf) && buf.eof(); if (!parsed || (min_block_num <= block_num && block_num <= max_block_num)) - to_delete_futures.emplace_back(path, zookeeper.asyncTryRemove(path)); + ops.emplace_back(zkutil::makeRemoveRequest(path, -1)); } +} - for (auto & pair : to_delete_futures) +void StorageReplicatedMergeTree::clearBlocksInPartition( + zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num) +{ + Coordination::Requests delete_requests; + getClearBlocksInPartitionOps(delete_requests, zookeeper, partition_id, min_block_num, max_block_num); + Coordination::Responses delete_responses; + auto code = zookeeper.tryMulti(delete_requests, delete_responses); + if (code != Coordination::Error::ZOK) { - const String & path = pair.first; - Coordination::Error rc = pair.second.get().error; - if (rc == Coordination::Error::ZNOTEMPTY) - { - /// Can happen if there are leftover block nodes with children created by previous server versions. - zookeeper.removeRecursive(path); - } - else if (rc != Coordination::Error::ZOK) - LOG_WARNING(log, "Error while deleting ZooKeeper path `{}`: {}, ignoring.", path, Coordination::errorMessage(rc)); + for (size_t i = 0; i < delete_requests.size(); ++i) + if (delete_responses[i]->error != Coordination::Error::ZOK) + LOG_WARNING(log, "Error while deleting ZooKeeper path `{}`: {}, ignoring.", delete_requests[i]->getPath(), Coordination::errorMessage(delete_responses[i]->error)); } - LOG_TRACE(log, "Deleted {} deduplication block IDs in partition ID {}", to_delete_futures.size(), partition_id); + LOG_TRACE(log, "Deleted {} deduplication block IDs in partition ID {}", delete_requests.size(), partition_id); } void StorageReplicatedMergeTree::replacePartitionFrom( @@ -6053,22 +6052,22 @@ bool StorageReplicatedMergeTree::dropPart( { ReplicatedMergeTreeMergePredicate merge_pred = queue.getMergePredicate(zookeeper); - /// TODO(nv) It is possible that part does not exist on replica which executes this statement. - /// Also, it possible for the part to not exist on any replicas, replica which created log entries for the part disappeared. - auto part = data_parts_by_info.find(part_info); - if (part == data_parts_by_info.end()) - throw Exception("Part " + part_name + " not found locally, won't try to drop it.", ErrorCodes::NOT_IMPLEMENTED); + auto part = getPartIfExists(part_info, {MergeTreeDataPartState::Committed}); - /// TODO(nv) get ops and commit together w/ log entry? - clearBlocksInPartition(*zookeeper, part_info.partition_id, part_info.min_block, part_info.max_block); + if (!part) + throw Exception("Part " + part_name + " not found locally, won't try to drop it.", ErrorCodes::NOT_IMPLEMENTED); /// There isn't a lot we can do otherwise. Can't cancel merges because it is possible that a replica already /// finished the merge. - if (partIsAssignedToBackgroundOperation(*part)) + if (partIsAssignedToBackgroundOperation(part)) throw Exception("Part " + part_name + " is currently participating in a background operation (mutation/merge)" + ", try again later.", ErrorCodes::PART_IS_TEMPORARILY_LOCKED); + Coordination::Requests ops; + getClearBlocksInPartitionOps(ops, *zookeeper, part_info.partition_id, part_info.min_block, part_info.max_block); + size_t clean_block_ops_size = ops.size(); + /// If `part_name` is result of a recent merge and source parts are still available then /// DROP_RANGE with detach will move this part together with source parts to `detached/` dir. entry.type = LogEntry::DROP_RANGE; @@ -6077,7 +6076,6 @@ bool StorageReplicatedMergeTree::dropPart( entry.detach = detach; entry.create_time = time(nullptr); - Coordination::Requests ops; ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/log", merge_pred.getVersion())); /// Make sure no new events were added to the log. ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); /// Just update version. @@ -6092,7 +6090,7 @@ bool StorageReplicatedMergeTree::dropPart( else zkutil::KeeperMultiException::check(rc, ops, responses); - String log_znode_path = dynamic_cast(*responses[1]).path_created; + String log_znode_path = dynamic_cast(*responses[clean_block_ops_size + 1]).path_created; entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); return true; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index bf02b6ae05b..2fc7848d87b 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -535,6 +535,7 @@ private: std::mutex existing_nodes_cache_mutex; bool existsNodeCached(const std::string & path); + void getClearBlocksInPartitionOps(Coordination::Requests & ops, zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num); /// Remove block IDs from `blocks/` in ZooKeeper for the given partition ID in the given block number range. void clearBlocksInPartition( zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num); From 347782966b14c5c97fef505383a8b12371cb9181 Mon Sep 17 00:00:00 2001 From: Fabiano Francesconi Date: Mon, 2 Nov 2020 18:36:54 +0100 Subject: [PATCH 115/314] Added Teralytics to adopters.md Added Teralytics to list of adopters. --- docs/en/introduction/adopters.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 24705a52a9a..d08e7833b33 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -90,6 +90,7 @@ toc_title: Adopters | Splunk | Business Analytics | Main product | — | — | [Slides in English, January 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup12/splunk.pdf) | | Spotify | Music | Experimentation | — | — | [Slides, July 2018](https://www.slideshare.net/glebus/using-clickhouse-for-experimentation-104247173) | | Staffcop | Information Security | Main Product | — | — | [Official website, Documentation](https://www.staffcop.ru/sce43) | +| Teralytics | Mobility | Analytics | — | — | [Tech blog](https://www.teralytics.net/knowledge-hub/visualizing-mobility-data-the-scalability-challenge) | | Tencent | Big Data | Data processing | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/5.%20ClickHouse大数据集群应用_李俊飞腾讯网媒事业部.pdf) | | Tencent | Messaging | Logging | — | — | [Talk in Chinese, November 2019](https://youtu.be/T-iVQRuw-QY?t=5050) | | Traffic Stars | AD network | — | — | — | [Slides in Russian, May 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup15/lightning/ninja.pdf) | From 3be8a56f5cdad9f1b514127a93f97f82509edda7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Oct 2020 22:28:46 +0300 Subject: [PATCH 116/314] Add log_queries_min_query_duration_ms Only queries slower then the value of this setting will go to system.query_log, i.e. something like slow_query_log in mysql. v2: log_queries_min_time renamed to log_queries_min_query_duration_ms v3: add current_database into system.query_thread_log v4: rewrite test using current_database v5: fix query_duration_ms in system.query_thread_log --- docs/en/operations/settings/settings.md | 15 ++++++ src/Common/ThreadStatus.h | 2 +- src/Core/Settings.h | 1 + src/Interpreters/ThreadStatusExt.cpp | 18 ++++--- src/Interpreters/executeQuery.cpp | 23 +++++--- ...og_queries_min_query_duration_ms.reference | 4 ++ ...1546_log_queries_min_query_duration_ms.sql | 54 +++++++++++++++++++ 7 files changed, 102 insertions(+), 15 deletions(-) create mode 100644 tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.reference create mode 100644 tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index a4bd7d77bfc..359a3579e46 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -680,6 +680,21 @@ Example: log_queries=1 ``` +## log_queries_min_query_duration_ms {#settings-log-queries-min-query-duration-ms} + +Minimal time for the query to run to get to the following tables: + +- `system.query_log` +- `system.query_thread_log` + +Only the queries with the following type will get to the log: + +- `QUERY_FINISH` +- `EXCEPTION_WHILE_PROCESSING` + +- Type: milliseconds +- Default value: 0 (any query) + ## log_queries_min_type {#settings-log-queries-min-type} `query_log` minimal type to log. diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 820ea449d66..1b4d20e9721 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -172,7 +172,7 @@ protected: void finalizeQueryProfiler(); - void logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database); + void logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database, std::chrono::time_point now); void assertState(const std::initializer_list & permitted_states, const char * description = nullptr) const; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 896fcaaca82..dabd3ed8b45 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -150,6 +150,7 @@ class IColumn; \ M(Bool, log_queries, 1, "Log requests and write the log to the system table.", 0) \ M(LogQueriesType, log_queries_min_type, QueryLogElementType::QUERY_START, "Minimal type in query_log to log, possible values (from low to high): QUERY_START, QUERY_FINISH, EXCEPTION_BEFORE_START, EXCEPTION_WHILE_PROCESSING.", 0) \ + M(Milliseconds, log_queries_min_query_duration_ms, 0, "Minimal time for the query to run, to get to the query_log/query_thread_log.", 0) \ M(UInt64, log_queries_cut_to_length, 100000, "If query length is greater than specified threshold (in bytes), then cut query when writing to query log. Also limit length of printed query in ordinary text log.", 0) \ \ M(DistributedProductMode, distributed_product_mode, DistributedProductMode::DENY, "How are distributed subqueries performed inside IN or JOIN sections?", IMPORTANT) \ diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index ffb9e140ce6..adb9a38b10d 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -242,8 +242,15 @@ void ThreadStatus::finalizePerformanceCounters() { const auto & settings = query_context->getSettingsRef(); if (settings.log_queries && settings.log_query_threads) - if (auto thread_log = global_context->getQueryThreadLog()) - logToQueryThreadLog(*thread_log, query_context->getCurrentDatabase()); + { + const auto now = std::chrono::system_clock::now(); + Int64 query_duration_ms = (time_in_microseconds(now) - query_start_time_microseconds) / 1000; + if (query_duration_ms >= settings.log_queries_min_query_duration_ms.totalMilliseconds()) + { + if (auto thread_log = global_context->getQueryThreadLog()) + logToQueryThreadLog(*thread_log, query_context->getCurrentDatabase(), now); + } + } } } catch (...) @@ -322,15 +329,14 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) #endif } -void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database) +void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database, std::chrono::time_point now) { QueryThreadLogElement elem; // construct current_time and current_time_microseconds using the same time point // so that the two times will always be equal up to a precision of a second. - const auto now = std::chrono::system_clock::now(); - auto current_time = time_in_seconds(now); - auto current_time_microseconds = time_in_microseconds(now); + auto current_time = time_in_seconds(now); + auto current_time_microseconds = time_in_microseconds(now); elem.event_time = current_time; elem.event_time_microseconds = current_time_microseconds; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 5feff841ca9..52940a337cb 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -241,7 +241,7 @@ static void onExceptionBeforeStart(const String & query_for_logging, Context & c /// Update performance counters before logging to query_log CurrentThread::finalizePerformanceCounters(); - if (settings.log_queries && elem.type >= settings.log_queries_min_type) + if (settings.log_queries && elem.type >= settings.log_queries_min_type && !settings.log_queries_min_query_duration_ms.totalMilliseconds()) if (auto query_log = context.getQueryLog()) query_log->add(elem); @@ -552,7 +552,7 @@ static std::tuple executeQueryImpl( if (settings.log_query_settings) elem.query_settings = std::make_shared(context.getSettingsRef()); - if (elem.type >= settings.log_queries_min_type) + if (elem.type >= settings.log_queries_min_type && !settings.log_queries_min_query_duration_ms.totalMilliseconds()) { if (auto query_log = context.getQueryLog()) query_log->add(elem); @@ -588,8 +588,12 @@ static std::tuple executeQueryImpl( }; /// Also make possible for caller to log successful query finish and exception during execution. - auto finish_callback = [elem, &context, ast, log_queries, log_queries_min_type = settings.log_queries_min_type, - status_info_to_query_log] + auto finish_callback = [elem, &context, ast, + log_queries, + log_queries_min_type = settings.log_queries_min_type, + log_queries_min_query_duration_ms = settings.log_queries_min_query_duration_ms.totalMilliseconds(), + status_info_to_query_log + ] (IBlockInputStream * stream_in, IBlockOutputStream * stream_out, QueryPipeline * query_pipeline) mutable { QueryStatus * process_list_elem = context.getProcessListElement(); @@ -655,7 +659,7 @@ static std::tuple executeQueryImpl( elem.thread_ids = std::move(info.thread_ids); elem.profile_counters = std::move(info.profile_counters); - if (log_queries && elem.type >= log_queries_min_type) + if (log_queries && elem.type >= log_queries_min_type && Int64(elem.query_duration_ms) >= log_queries_min_query_duration_ms) { if (auto query_log = context.getQueryLog()) query_log->add(elem); @@ -694,8 +698,11 @@ static std::tuple executeQueryImpl( } }; - auto exception_callback = [elem, &context, ast, log_queries, log_queries_min_type = settings.log_queries_min_type, quota(quota), - status_info_to_query_log] () mutable + auto exception_callback = [elem, &context, ast, + log_queries, + log_queries_min_type = settings.log_queries_min_type, + log_queries_min_query_duration_ms = settings.log_queries_min_query_duration_ms.totalMilliseconds(), + quota(quota), status_info_to_query_log] () mutable { if (quota) quota->used(Quota::ERRORS, 1, /* check_exceeded = */ false); @@ -729,7 +736,7 @@ static std::tuple executeQueryImpl( logException(context, elem); /// In case of exception we log internal queries also - if (log_queries && elem.type >= log_queries_min_type) + if (log_queries && elem.type >= log_queries_min_type && Int64(elem.query_duration_ms) >= log_queries_min_query_duration_ms) { if (auto query_log = context.getQueryLog()) query_log->add(elem); diff --git a/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.reference b/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.reference new file mode 100644 index 00000000000..0463db26710 --- /dev/null +++ b/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.reference @@ -0,0 +1,4 @@ +0 +0 +1 +1 diff --git a/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql b/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql new file mode 100644 index 00000000000..f0f681288cf --- /dev/null +++ b/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql @@ -0,0 +1,54 @@ +set log_queries_min_query_duration_ms=300000; +set log_query_threads=1; +set log_queries=1; + +-- +-- fast -- no logging +-- +select '01546_log_queries_min_query_duration_ms-fast' format Null; +system flush logs; + +-- No logging, since the query is fast enough. +select count() +from system.query_log +where + query like '%01546_log_queries_min_query_duration_ms-fast%' + and query not like '%system.query_log%' + and current_database = currentDatabase() + and event_date = today() + and event_time >= now() - interval 1 minute; +select count() +from system.query_thread_log +where + query like '%01546_log_queries_min_query_duration_ms-fast%' + and query not like '%system.query_thread_log%' + and current_database = currentDatabase() + and event_date = today() + and event_time >= now() - interval 1 minute; + +-- +-- slow -- query logged +-- +set log_queries_min_query_duration_ms=300; +select '01546_log_queries_min_query_duration_ms-slow', sleep(0.4) format Null; +system flush logs; + +-- With the limit on minimum execution time, "query start" and "exception before start" events are not logged, only query finish. +select count() +from system.query_log +where + query like '%01546_log_queries_min_query_duration_ms-slow%' + and query not like '%system.query_log%' + and current_database = currentDatabase() + and event_date = today() + and event_time >= now() - interval 1 minute; +-- There at least two threads involved in a simple query +-- (one thread just waits another, sigh) +select count() == 2 +from system.query_thread_log +where + query like '%01546_log_queries_min_query_duration_ms-slow%' + and query not like '%system.query_thread_log%' + and current_database = currentDatabase() + and event_date = today() + and event_time >= now() - interval 1 minute; From e6d8ab2270532e3ed5171d9ee1e8b52ca2843f82 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 2 Nov 2020 21:37:23 +0300 Subject: [PATCH 117/314] Fix possible name collision in ALTER --- src/Interpreters/InterpreterCreateQuery.cpp | 3 ++- src/Storages/AlterCommands.cpp | 9 ++++----- .../0_stateless/01552_alter_name_collision.reference | 0 tests/queries/0_stateless/01552_alter_name_collision.sql | 3 +++ 4 files changed, 9 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/01552_alter_name_collision.reference create mode 100644 tests/queries/0_stateless/01552_alter_name_collision.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ddb1d738031..6a8bdbea1ec 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -362,7 +363,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( if (col_decl.type) { const auto & final_column_name = col_decl.name; - const auto tmp_column_name = final_column_name + "_tmp"; + const auto tmp_column_name = final_column_name + "_tmp_alter" + toString(randomSeed()); const auto * data_type_ptr = column_names_and_types.back().type.get(); default_expr_list->children.emplace_back( diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 8cae7866748..559121d6911 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -22,12 +22,11 @@ #include #include #include +#include #include #include #include - - -#include +#include namespace DB @@ -1117,7 +1116,7 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con data_type_ptr = command.data_type; const auto & final_column_name = column_name; - const auto tmp_column_name = final_column_name + "_tmp"; + const auto tmp_column_name = final_column_name + "_tmp_alter" + toString(randomSeed()); default_expr_list->children.emplace_back(setAlias( addTypeConversionToAST(std::make_shared(tmp_column_name), data_type_ptr->getName()), @@ -1133,7 +1132,7 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con continue; const auto & final_column_name = column_name; - const auto tmp_column_name = final_column_name + "_tmp"; + const auto tmp_column_name = final_column_name + "_tmp_alter" + toString(randomSeed()); const auto data_type_ptr = command.data_type; default_expr_list->children.emplace_back(setAlias( diff --git a/tests/queries/0_stateless/01552_alter_name_collision.reference b/tests/queries/0_stateless/01552_alter_name_collision.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01552_alter_name_collision.sql b/tests/queries/0_stateless/01552_alter_name_collision.sql new file mode 100644 index 00000000000..dc717f1071a --- /dev/null +++ b/tests/queries/0_stateless/01552_alter_name_collision.sql @@ -0,0 +1,3 @@ +DROP TABLE IF EXISTS test; +CREATE TABLE test(test String DEFAULT 'test', test_tmp Int DEFAULT 1)ENGINE = Memory; +DROP TABLE test; From 08b63fde77df7f858a667caccdca99dda9dffa10 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Mon, 2 Nov 2020 21:39:54 +0300 Subject: [PATCH 118/314] done --- src/Interpreters/ExpressionActions.cpp | 5 ++++- .../0_stateless/01548_with_totals_having.reference | 12 ++++++++++++ .../queries/0_stateless/01548_with_totals_having.sql | 2 ++ 3 files changed, 18 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01548_with_totals_having.reference create mode 100644 tests/queries/0_stateless/01548_with_totals_having.sql diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 762ad6ae575..f5cd0ff409d 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -50,6 +50,7 @@ namespace ErrorCodes extern const int TOO_MANY_TEMPORARY_COLUMNS; extern const int TOO_MANY_TEMPORARY_NON_CONST_COLUMNS; extern const int TYPE_MISMATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } /// Read comment near usage @@ -375,7 +376,6 @@ void ExpressionAction::execute(Block & block, bool dry_run) const auto source = block.getByName(source_name); block.erase(source_name); source.column = source.column->convertToFullColumnIfConst(); - const ColumnArray * array = typeid_cast(source.column.get()); if (!array) throw Exception("ARRAY JOIN of not array: " + source_name, ErrorCodes::TYPE_MISMATCH); @@ -387,6 +387,9 @@ void ExpressionAction::execute(Block & block, bool dry_run) const source.type = assert_cast(*source.type).getNestedType(); source.name = result_name; + if (source.type->getTypeId() == TypeIndex::Nothing) + throw Exception("ARRAY JOIN of empty array is not allowed", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + block.insert(std::move(source)); break; diff --git a/tests/queries/0_stateless/01548_with_totals_having.reference b/tests/queries/0_stateless/01548_with_totals_having.reference new file mode 100644 index 00000000000..c728434f5dd --- /dev/null +++ b/tests/queries/0_stateless/01548_with_totals_having.reference @@ -0,0 +1,12 @@ +0 +0 +0 +0 +1 +1 +1 +2 +2 +3 + +0 diff --git a/tests/queries/0_stateless/01548_with_totals_having.sql b/tests/queries/0_stateless/01548_with_totals_having.sql new file mode 100644 index 00000000000..167d879bbeb --- /dev/null +++ b/tests/queries/0_stateless/01548_with_totals_having.sql @@ -0,0 +1,2 @@ +SELECT * FROM numbers(4) GROUP BY number WITH TOTALS HAVING sum(number) <= arrayJoin([]); -- { serverError 43 } +SELECT * FROM numbers(4) GROUP BY number WITH TOTALS HAVING sum(number) <= arrayJoin([3, 2, 1, 0]) ORDER BY number; From 3bc8ff6af6b2e8ccbbb1b74f2819bf6d9e266643 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 2 Nov 2020 21:52:04 +0300 Subject: [PATCH 119/314] Add a test for #1148 --- .../01552_dict_fixedstring.reference | 2 ++ .../0_stateless/01552_dict_fixedstring.sql | 20 +++++++++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/01552_dict_fixedstring.reference create mode 100644 tests/queries/0_stateless/01552_dict_fixedstring.sql diff --git a/tests/queries/0_stateless/01552_dict_fixedstring.reference b/tests/queries/0_stateless/01552_dict_fixedstring.reference new file mode 100644 index 00000000000..2f877f0c754 --- /dev/null +++ b/tests/queries/0_stateless/01552_dict_fixedstring.reference @@ -0,0 +1,2 @@ + +Hello\0World diff --git a/tests/queries/0_stateless/01552_dict_fixedstring.sql b/tests/queries/0_stateless/01552_dict_fixedstring.sql new file mode 100644 index 00000000000..7e0269f8e50 --- /dev/null +++ b/tests/queries/0_stateless/01552_dict_fixedstring.sql @@ -0,0 +1,20 @@ +DROP TABLE IF EXISTS src; + +CREATE TABLE src (k UInt64, s FixedString(11)) ENGINE = Memory; +INSERT INTO src VALUES (1, 'Hello\0World'); + +DROP DICTIONARY IF EXISTS dict; +CREATE DICTIONARY dict +( + k UInt64, + s String +) +PRIMARY KEY k +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER default TABLE 'src')) +LAYOUT(FLAT) +LIFETIME(MIN 10 MAX 10); + +SELECT dictGet(currentDatabase() || '.dict', 's', number) FROM numbers(2); + +DROP TABLE src; +DROP DICTIONARY dict; From bdd453c54dcc78638c4bfe2e36af5c1ee577301c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 3 Nov 2020 00:47:45 +0300 Subject: [PATCH 120/314] Add 01546_log_queries_min_query_duration_ms into arcadia_skip_list In attempt to fix "Yandex synchronization check (only for Yandex employees)" --- tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index f5b81c08520..f6d899a5742 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -155,5 +155,6 @@ 01509_dictionary_preallocate 01526_max_untracked_memory 01530_drop_database_atomic_sync +01546_log_queries_min_query_duration_ms 01547_query_log_current_database 01548_query_log_query_execution_ms From a25996ed8d44743608932725f5a87421202ed9cf Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 3 Nov 2020 02:10:19 +0300 Subject: [PATCH 121/314] Upload perf test results to the CI database --- docker/test/performance-comparison/compare.sh | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index d0cf12baa9e..20720e0b976 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1074,6 +1074,56 @@ wait unset IFS } +function upload_results +{ + if ! [ -v CHPC_DATABASE_URL ] + then + echo Database for test results is not specified, will not upload them. + return 0 + fi + + # Surprisingly, clickhouse-client doesn't understand --host 127.0.0.1:9000 + # so I have to do this instead. I tried to use Poco URI parser for this, + # but it's also broken and can't parse host:port. + IFS=':' read host port <<<"${CHPC_DATABASE_URL}" + + upload_client=(clickhouse-client + --host "${host}" + --port "${port}" + --secure + --user "${CHPC_DATABASE_USER}" + --password "${CHPC_DATABASE_PASSWORD}" + --config "ch/tests/config/client_config.xml" + --database perftest + -m + --date_time_input_format=best_effort) + + set +x # Don't show password in the log + cat "report/all-query-metrics.tsv" | "${upload_client[@]}" --query " + insert into query_metrics_tmp + select + toDate(event_time) event_date, + toDateTime('$(cd ch && git show -s --format=%ci "$SHA_TO_TEST" | cut -d' ' -f-2)') event_time, + $PR_TO_TEST pr_number, + '$REF_SHA' old_sha, + '$SHA_TO_TEST' new_sha, + test, + query_index, + query_display_name, + metric_name, + old_value, + new_value, + diff, + stat_threshold + from input('metric_name text, old_value float, new_value float, diff float, + ratio_display_text text, stat_threshold float, + test text, query_index int, query_display_name text') + settings date_time_input_format='best_effort' + format TSV + settings date_time_input_format='best_effort'" + set -x +} + # Check that local and client are in PATH clickhouse-local --version > /dev/null clickhouse-client --version > /dev/null @@ -1145,6 +1195,9 @@ case "$stage" in time "$script_dir/report.py" --report=all-queries > all-queries.html 2> >(tee -a report/errors.log 1>&2) ||: time "$script_dir/report.py" > report.html ;& +"upload_results") + time upload_results ||: + ;& esac # Print some final debug info to help debug Weirdness, of which there is plenty. From da8ac948bfb1b329bcc3d479981177ad0b2344f2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 3 Nov 2020 12:24:10 +0300 Subject: [PATCH 122/314] Disallow to drop parts with parallel quorum --- src/Storages/StorageMergeTree.cpp | 7 +- src/Storages/StorageReplicatedMergeTree.cpp | 94 ++++++------------- src/Storages/StorageReplicatedMergeTree.h | 6 +- .../0_stateless/01451_detach_drop_part.sql | 2 + ...eplicated_detach_drop_and_quorum.reference | 5 + ...1451_replicated_detach_drop_and_quorum.sql | 49 ++++++++++ .../01451_replicated_detach_drop_part.sql | 2 + 7 files changed, 98 insertions(+), 67 deletions(-) create mode 100644 tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.reference create mode 100644 tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.sql diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 844884714ab..8f32d784055 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -42,6 +42,7 @@ namespace ErrorCodes extern const int CANNOT_ASSIGN_OPTIMIZE; extern const int TIMEOUT_EXCEEDED; extern const int UNKNOWN_POLICY; + extern const int NO_SUCH_DATA_PART; } namespace ActionLocks @@ -1229,7 +1230,6 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool MergeTreeData::DataPartsVector parts_to_remove; - /// TODO: should we include PreComitted parts like in Replicated case? if (drop_part) { String part_name = partition->as().value.safeGet(); @@ -1237,7 +1237,10 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool if (part) parts_to_remove.push_back(part); - } else + else + throw Exception("Part " + part_name + " not found, won't try to drop it.", ErrorCodes::NO_SUCH_DATA_PART); + } + else { String partition_id = getPartitionIDFromQuery(partition, context); parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a8e2df4e8f9..2f3d3ae1010 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -115,6 +115,7 @@ namespace ErrorCodes extern const int DIRECTORY_ALREADY_EXISTS; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int UNKNOWN_POLICY; + extern const int NO_SUCH_DATA_PART; } namespace ActionLocks @@ -3275,61 +3276,34 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id) } } -void StorageReplicatedMergeTree::updateLastPartNodeIfMatches(const String & partition_id, const String & old_part_name, const String & new_part_name) + +bool StorageReplicatedMergeTree::partIsInsertingWithParallelQuorum(const MergeTreePartInfo & part_info) const +{ + auto zookeeper = getZooKeeper(); + return zookeeper->exists(zookeeper_path + "/quorum/parallel/" + part_info.getPartName()); +} + +bool StorageReplicatedMergeTree::partIsLastQuorumPart(const MergeTreePartInfo & part_info) const { auto zookeeper = getZooKeeper(); - const String quorum_last_part_path = zookeeper_path + "/quorum/last_part"; + const String parts_with_quorum_path = zookeeper_path + "/quorum/last_part"; - while (true) - { - Coordination::Stat added_parts_stat; - String old_added_parts = zookeeper->get(quorum_last_part_path, &added_parts_stat); + String parts_with_quorum_str = zookeeper->get(parts_with_quorum_path); - ReplicatedMergeTreeQuorumAddedParts parts_with_quorum(format_version); + if (parts_with_quorum_str.empty()) + return false; - if (!old_added_parts.empty()) - parts_with_quorum.fromString(old_added_parts); + ReplicatedMergeTreeQuorumAddedParts parts_with_quorum(format_version); + parts_with_quorum.fromString(parts_with_quorum_str); - if (!parts_with_quorum.added_parts.count(partition_id)) - { - /// There is no information about partition at all. - break; - } + auto partition_it = parts_with_quorum.added_parts.find(part_info.partition_id); + if (partition_it == parts_with_quorum.added_parts.end()) + return false; - /// Part for which last quorum was reached in partition_id. - auto quorum_part_info = MergeTreePartInfo::fromPartName(parts_with_quorum.added_parts.at(partition_id), format_version); - auto old_part_info = MergeTreePartInfo::fromPartName(old_part_name, format_version); - - /// Update last part for which quorum was reached. - if (old_part_info.contains(quorum_part_info)) - parts_with_quorum.added_parts.emplace(partition_id, new_part_name); - - /// Serialize and try update. - String new_added_parts = parts_with_quorum.toString(); - - auto code = zookeeper->trySet(quorum_last_part_path, new_added_parts, added_parts_stat.version); - - if (code == Coordination::Error::ZOK) - { - break; - } - else if (code == Coordination::Error::ZNONODE) - { - /// Node is deleted. It is impossible, but it is Ok. - break; - } - else if (code == Coordination::Error::ZBADVERSION) - { - /// Node was updated meanwhile. We must re-read it and repeat all the actions. - continue; - } - else - throw Coordination::Exception(code, quorum_last_part_path); - } + return partition_it->second == part_info.getPartName(); } - bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const StorageMetadataPtr & metadata_snapshot, const String & source_replica_path, bool to_detached, size_t quorum, zkutil::ZooKeeper::Ptr zookeeper_) { @@ -4361,23 +4335,7 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool de } } - bool drop_entire_partition = !drop_part; - - /// Cleaning possibly stored information about parts from /quorum/last_part node in ZooKeeper. - if (drop_part) - { - auto part_info = MergeTreePartInfo::fromPartName(partition->as().value.safeGet(), format_version); - auto data_parts_vec = getDataPartsVectorInPartition(DataPartState::Committed, part_info.partition_id); - std::sort(data_parts_vec.begin(), data_parts_vec.end(), LessDataPart()); - - auto prev_part = std::upper_bound(data_parts_vec.begin(), data_parts_vec.end(), part_info, LessDataPart()); - if (prev_part != data_parts_vec.end()) - updateLastPartNodeIfMatches(part_info.partition_id, part_info.getPartName(), (*prev_part)->info.getPartName()); - else if (data_parts_vec.empty()) - drop_entire_partition = true; - } - - if (drop_entire_partition) + if (!drop_part) { String partition_id = getPartitionIDFromQuery(partition, query_context); cleanLastPartNode(partition_id); @@ -6058,14 +6016,22 @@ bool StorageReplicatedMergeTree::dropPart( auto part = getPartIfExists(part_info, {MergeTreeDataPartState::Committed}); if (!part) - throw Exception("Part " + part_name + " not found locally, won't try to drop it.", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("Part " + part_name + " not found locally, won't try to drop it.", ErrorCodes::NO_SUCH_DATA_PART); /// There isn't a lot we can do otherwise. Can't cancel merges because it is possible that a replica already /// finished the merge. if (partIsAssignedToBackgroundOperation(part)) throw Exception("Part " + part_name + " is currently participating in a background operation (mutation/merge)" - + ", try again later.", ErrorCodes::PART_IS_TEMPORARILY_LOCKED); + + ", try again later", ErrorCodes::PART_IS_TEMPORARILY_LOCKED); + + if (partIsLastQuorumPart(part->info)) + throw Exception("Part " + part_name + " is last inserted part with quorum in partition. Cannot drop", + ErrorCodes::NOT_IMPLEMENTED); + + if (partIsInsertingWithParallelQuorum(part->info)) + throw Exception("Part " + part_name + " is inserting with parallel quorum. Cannot drop", + ErrorCodes::NOT_IMPLEMENTED); Coordination::Requests ops; getClearBlocksInPartitionOps(ops, *zookeeper, part_info.partition_id, part_info.min_block, part_info.max_block); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 2fc7848d87b..b72c3344f3e 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -502,7 +502,11 @@ private: /// Deletes info from quorum/last_part node for particular partition_id. void cleanLastPartNode(const String & partition_id); - void updateLastPartNodeIfMatches(const String & partition_id, const String & old_part_name, const String & new_part_name); + /// Part name is stored in quorum/last_part for corresponding partition_id. + bool partIsLastQuorumPart(const MergeTreePartInfo & part_info) const; + + /// Part currently inserting with quorum (node quorum/parallel/part_name exists) + bool partIsInsertingWithParallelQuorum(const MergeTreePartInfo & part_info) const; /// Creates new block number if block with such block_id does not exist std::optional allocateBlockNumber( diff --git a/tests/queries/0_stateless/01451_detach_drop_part.sql b/tests/queries/0_stateless/01451_detach_drop_part.sql index 47e61f2d924..7a2815f9a3e 100644 --- a/tests/queries/0_stateless/01451_detach_drop_part.sql +++ b/tests/queries/0_stateless/01451_detach_drop_part.sql @@ -9,6 +9,8 @@ INSERT INTO mt VALUES (2); SELECT v FROM mt ORDER BY v; +ALTER TABLE mt DETACH PART 'all_100_100_0'; -- { serverError 232 } + ALTER TABLE mt DETACH PART 'all_2_2_0'; SELECT v FROM mt ORDER BY v; diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.reference b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.reference new file mode 100644 index 00000000000..3768ac7b852 --- /dev/null +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.reference @@ -0,0 +1,5 @@ +all_0_0_0 +all_2_2_0 +1 +all_2_2_0 +1 diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.sql new file mode 100644 index 00000000000..fca14b81e27 --- /dev/null +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum.sql @@ -0,0 +1,49 @@ +SET replication_alter_partitions_sync = 2; + + +DROP TABLE IF EXISTS replica1; +DROP TABLE IF EXISTS replica2; + +CREATE TABLE replica1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01451/quorum', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; +CREATE TABLE replica2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01451/quorum', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; + +INSERT INTO replica1 VALUES (0); + +SYSTEM SYNC REPLICA replica2; + +SELECT name FROM system.parts WHERE table = 'replica2' and database = currentDatabase() and active = 1; + +ALTER TABLE replica2 DETACH PART 'all_0_0_0'; + +SELECT * FROM replica1; + +SELECT * FROM replica2; + +-- drop of empty partition works +ALTER TABLE replica2 DROP PARTITION ID 'all'; + +SET insert_quorum=2; + +INSERT INTO replica2 VALUES (1); + +SYSTEM SYNC REPLICA replica2; + +ALTER TABLE replica1 DETACH PART 'all_2_2_0'; --{serverError 48} + +SELECT name FROM system.parts WHERE table = 'replica1' and database = currentDatabase() and active = 1 ORDER BY name; + +SELECT COUNT() FROM replica1; + +SET insert_quorum_parallel=1; + +INSERT INTO replica2 VALUES (2); + +-- should work, parallel quorum nodes exists only during insert +ALTER TABLE replica1 DROP PART 'all_3_3_0'; + +SELECT name FROM system.parts WHERE table = 'replica1' and database = currentDatabase() and active = 1 ORDER BY name; + +SELECT COUNT() FROM replica1; + +DROP TABLE IF EXISTS replica1; +DROP TABLE IF EXISTS replica2; diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_part.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_part.sql index 1209f11b68e..3cd9fc7bc7e 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_part.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_part.sql @@ -10,6 +10,8 @@ INSERT INTO replica1 VALUES (0); INSERT INTO replica1 VALUES (1); INSERT INTO replica1 VALUES (2); +ALTER TABLE replica1 DETACH PART 'all_100_100_0'; -- { serverError 232 } + SELECT v FROM replica1 ORDER BY v; SYSTEM SYNC REPLICA replica2; From 72bde1c688f61ea77ca530e6422445507d2bbea6 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 3 Nov 2020 17:32:31 +0800 Subject: [PATCH 123/314] better change the config --- .../integration/test_reload_zookeeper/test.py | 35 +++---------------- 1 file changed, 5 insertions(+), 30 deletions(-) diff --git a/tests/integration/test_reload_zookeeper/test.py b/tests/integration/test_reload_zookeeper/test.py index 35958c95417..1a48535d3fd 100644 --- a/tests/integration/test_reload_zookeeper/test.py +++ b/tests/integration/test_reload_zookeeper/test.py @@ -28,28 +28,6 @@ def start_cluster(): yield cluster finally: - ## write back the configs - config = open(ZK_CONFIG_PATH, 'w') - config.write( -""" - - - - zoo1 - 2181 - - - zoo2 - 2181 - - - zoo3 - 2181 - - 2000 - - - """) config.close() cluster.shutdown() @@ -71,8 +49,7 @@ def test_reload_zookeeper(start_cluster): node.query("INSERT INTO test_table(date, id) select today(), number FROM numbers(1000)") ## remove zoo2, zoo3 from configs - config = open(ZK_CONFIG_PATH, 'w') - config.write( + new_config = """ @@ -84,8 +61,7 @@ def test_reload_zookeeper(start_cluster): """ - ) - config.close() + node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) ## config reloads, but can still work assert_eq_with_retry(node, "SELECT COUNT() FROM test_table", '1000', retry_count=120, sleep_time=0.5) @@ -101,8 +77,7 @@ def test_reload_zookeeper(start_cluster): node.query("SELECT COUNT() FROM test_table") ## set config to zoo2, server will be normal - config = open(ZK_CONFIG_PATH, 'w') - config.write( + new_config = """ @@ -114,7 +89,7 @@ def test_reload_zookeeper(start_cluster): """ - ) - config.close() + node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) + assert_eq_with_retry(node, "SELECT COUNT() FROM test_table", '1000', retry_count=120, sleep_time=0.5) From af5ecbef278ff2d86e0a75d1a7e9e6bcfe92983b Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 3 Nov 2020 17:57:24 +0800 Subject: [PATCH 124/314] set zookeeper config --- src/Interpreters/Context.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7f2ada8a426..be701ff171b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -363,6 +363,7 @@ struct ContextShared /// Initialized on demand (on distributed storages initialization) since Settings should be initialized std::unique_ptr clusters; ConfigurationPtr clusters_config; /// Stores updated configs + ConfigurationPtr zookeeper_config; /// Stores zookeeperd configs mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config #if USE_EMBEDDED_COMPILER @@ -1472,8 +1473,9 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const { std::lock_guard lock(shared->zookeeper_mutex); + const auto & config = shared->zookeeper_config ? *shared->zookeeper_config : getConfigRef(); if (!shared->zookeeper) - shared->zookeeper = std::make_shared(getConfigRef(), "zookeeper"); + shared->zookeeper = std::make_shared(config, "zookeeper"); else if (shared->zookeeper->expired()) shared->zookeeper = shared->zookeeper->startNewSession(); @@ -1507,6 +1509,8 @@ void Context::resetZooKeeper() const void Context::reloadZooKeeperIfChanged(const ConfigurationPtr & config) const { std::lock_guard lock(shared->zookeeper_mutex); + shared->zookeeper_config = config; + if (!shared->zookeeper || shared->zookeeper->configChanged(*config, "zookeeper")) { shared->zookeeper = std::make_shared(*config, "zookeeper"); From 9cfeb9fd324f26362e894d22083a1e40f80df329 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 3 Nov 2020 18:13:48 +0800 Subject: [PATCH 125/314] remove unused codes --- tests/integration/test_reload_zookeeper/test.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/integration/test_reload_zookeeper/test.py b/tests/integration/test_reload_zookeeper/test.py index 1a48535d3fd..ab0f3491ae9 100644 --- a/tests/integration/test_reload_zookeeper/test.py +++ b/tests/integration/test_reload_zookeeper/test.py @@ -10,9 +10,6 @@ from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper.xml') node = cluster.add_instance('node', with_zookeeper=True) -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -ZK_CONFIG_PATH = os.path.join(SCRIPT_DIR, 'configs/zookeeper.xml') - @pytest.fixture(scope="module") def start_cluster(): @@ -28,7 +25,6 @@ def start_cluster(): yield cluster finally: - config.close() cluster.shutdown() def test_reload_zookeeper(start_cluster): From 2534ea342689f686d64ad64291eb85583c9cb31e Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Tue, 3 Nov 2020 10:22:51 +0000 Subject: [PATCH 126/314] fix bug in minMap and maxMap with zero values --- .../AggregateFunctionSumMap.h | 55 ++++++++++--------- .../01280_min_map_max_map.reference | 5 ++ .../0_stateless/01280_min_map_max_map.sql | 7 +++ 3 files changed, 42 insertions(+), 25 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index 456334ee9c3..33fd87c0a87 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -56,7 +56,7 @@ struct AggregateFunctionMapData * minMap and maxMap share the same idea, but calculate min and max correspondingly. */ -template +template class AggregateFunctionMapBase : public IAggregateFunctionDataHelper< AggregateFunctionMapData>, Derived> { @@ -255,23 +255,27 @@ public: { // Final step does compaction of keys that have zero values, this mutates the state auto & merged_maps = this->data(place).merged_maps; - for (auto it = merged_maps.cbegin(); it != merged_maps.cend();) - { - // Key is not compacted if it has at least one non-zero value - bool erase = true; - for (size_t col = 0; col < values_types.size(); ++col) - { - if (it->second[col] != values_types[col]->getDefault()) - { - erase = false; - break; - } - } - if (erase) - it = merged_maps.erase(it); - else - ++it; + if constexpr (compact) + { + for (auto it = merged_maps.cbegin(); it != merged_maps.cend();) + { + // Key is not compacted if it has at least one non-zero value + bool erase = true; + for (size_t col = 0; col < values_types.size(); ++col) + { + if (it->second[col] != values_types[col]->getDefault()) + { + erase = false; + break; + } + } + + if (erase) + it = merged_maps.erase(it); + else + ++it; + } } size_t size = merged_maps.size(); @@ -314,11 +318,11 @@ public: template class AggregateFunctionSumMap final : - public AggregateFunctionMapBase, FieldVisitorSum, overflow, tuple_argument> + public AggregateFunctionMapBase, FieldVisitorSum, overflow, tuple_argument, true> { private: using Self = AggregateFunctionSumMap; - using Base = AggregateFunctionMapBase; + using Base = AggregateFunctionMapBase; public: AggregateFunctionSumMap(const DataTypePtr & keys_type_, @@ -342,11 +346,12 @@ class AggregateFunctionSumMapFiltered final : AggregateFunctionSumMapFiltered, FieldVisitorSum, overflow, - tuple_argument> + tuple_argument, + true> { private: using Self = AggregateFunctionSumMapFiltered; - using Base = AggregateFunctionMapBase; + using Base = AggregateFunctionMapBase; /// ARCADIA_BUILD disallow unordered_set for big ints for some reason static constexpr const bool allow_hash = !OverBigInt; @@ -474,11 +479,11 @@ public: template class AggregateFunctionMinMap final : - public AggregateFunctionMapBase, FieldVisitorMin, true, tuple_argument> + public AggregateFunctionMapBase, FieldVisitorMin, true, tuple_argument, false> { private: using Self = AggregateFunctionMinMap; - using Base = AggregateFunctionMapBase; + using Base = AggregateFunctionMapBase; public: AggregateFunctionMinMap(const DataTypePtr & keys_type_, @@ -498,11 +503,11 @@ public: template class AggregateFunctionMaxMap final : - public AggregateFunctionMapBase, FieldVisitorMax, true, tuple_argument> + public AggregateFunctionMapBase, FieldVisitorMax, true, tuple_argument, false> { private: using Self = AggregateFunctionMaxMap; - using Base = AggregateFunctionMapBase; + using Base = AggregateFunctionMapBase; public: AggregateFunctionMaxMap(const DataTypePtr & keys_type_, diff --git a/tests/queries/0_stateless/01280_min_map_max_map.reference b/tests/queries/0_stateless/01280_min_map_max_map.reference index dd707d602c7..abb500553fd 100644 --- a/tests/queries/0_stateless/01280_min_map_max_map.reference +++ b/tests/queries/0_stateless/01280_min_map_max_map.reference @@ -22,3 +22,8 @@ (['1970-01-01 03:00:01'],[2]) ([1.01],[2]) (['a'],[2]) +([1,2],[0,0]) +([1,2],[0,0]) +(['A','B'],['','']) +(['A','B'],['','']) +([],[]) diff --git a/tests/queries/0_stateless/01280_min_map_max_map.sql b/tests/queries/0_stateless/01280_min_map_max_map.sql index 02731eee601..61d4a8ee4c9 100644 --- a/tests/queries/0_stateless/01280_min_map_max_map.sql +++ b/tests/queries/0_stateless/01280_min_map_max_map.sql @@ -31,3 +31,10 @@ select maxMap(val, cnt) from values ('val Array(Date), cnt Array(Int16)', ([1], select maxMap(val, cnt) from values ('val Array(DateTime(\'Europe/Moscow\')), cnt Array(Int32)', ([1], [1]), ([1], [2])); select maxMap(val, cnt) from values ('val Array(Decimal(10, 2)), cnt Array(Int16)', (['1.01'], [1]), (['1.01'], [2])); select maxMap(val, cnt) from values ('val Array(Enum16(\'a\'=1)), cnt Array(Int16)', (['a'], [1]), (['a'], [2])); + +-- bugfix, minMap and maxMap should not remove values with zero and empty strings but this behavior should not affect sumMap +select minMap(val, cnt) from values ('val Array(UInt64), cnt Array(UInt64)', ([1], [0]), ([2], [0])); +select maxMap(val, cnt) from values ('val Array(UInt64), cnt Array(UInt64)', ([1], [0]), ([2], [0])); +select minMap(val, cnt) from values ('val Array(String), cnt Array(String)', (['A'], ['']), (['B'], [''])); +select minMap(val, cnt) from values ('val Array(String), cnt Array(String)', (['A'], ['']), (['B'], [''])); +select sumMap(val, cnt) from values ('val Array(UInt64), cnt Array(UInt64)', ([1], [0]), ([2], [0])); From 4d8fb3842d48047be16e758d58b1e2b44c203384 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 3 Nov 2020 13:24:51 +0300 Subject: [PATCH 127/314] Fix strange test --- tests/queries/0_stateless/01093_cyclic_defaults_filimonov.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01093_cyclic_defaults_filimonov.sql b/tests/queries/0_stateless/01093_cyclic_defaults_filimonov.sql index 548cd794ba3..f5f88db9d66 100644 --- a/tests/queries/0_stateless/01093_cyclic_defaults_filimonov.sql +++ b/tests/queries/0_stateless/01093_cyclic_defaults_filimonov.sql @@ -1,7 +1,7 @@ CREATE TABLE test ( - `a1` UInt64 DEFAULT a + 1, - `a1` UInt64 DEFAULT a + 1, + `a0` UInt64 DEFAULT a1 + 1, + `a1` UInt64 DEFAULT a0 + 1, `a2` UInt64 DEFAULT a3 + a4, `a3` UInt64 DEFAULT a2 + 1, `a4` UInt64 ALIAS a3 + 1 From 888440fe8f2ea474b3ff6ba145fe56e42a80bd6d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 3 Nov 2020 13:41:56 +0300 Subject: [PATCH 128/314] Skip test in Arcadia --- tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index f5b81c08520..eb9c65885d9 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -157,3 +157,4 @@ 01530_drop_database_atomic_sync 01547_query_log_current_database 01548_query_log_query_execution_ms +01552_dict_fixedstring From d95bcc8890c54b09f73b5a2aa3e03e578b745fab Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Mon, 2 Nov 2020 21:55:44 +0300 Subject: [PATCH 129/314] done --- src/Core/Field.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Core/Field.h b/src/Core/Field.h index 46d7713e1e4..66e4f0ac8db 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -768,8 +768,7 @@ T & Field::get() // Disregard signedness when converting between int64 types. constexpr Field::Types::Which target = TypeToEnum>::value; if (target != which - && (!isInt64FieldType(target) || !isInt64FieldType(which)) - && target != Field::Types::Decimal64 /* DateTime64 fields */) + && (!isInt64FieldType(target) || !isInt64FieldType(which))) throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid Field get from type {} to type {}", Types::toString(which), Types::toString(target)); #endif From dea4bd483ec443c449dab260c5a345fa92fcf8b5 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 3 Nov 2020 19:40:45 +0800 Subject: [PATCH 130/314] remove unused codes2 --- tests/integration/test_reload_zookeeper/test.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_reload_zookeeper/test.py b/tests/integration/test_reload_zookeeper/test.py index ab0f3491ae9..abc640c79dc 100644 --- a/tests/integration/test_reload_zookeeper/test.py +++ b/tests/integration/test_reload_zookeeper/test.py @@ -45,8 +45,7 @@ def test_reload_zookeeper(start_cluster): node.query("INSERT INTO test_table(date, id) select today(), number FROM numbers(1000)") ## remove zoo2, zoo3 from configs - new_config = -""" + new_config = """ @@ -73,8 +72,7 @@ def test_reload_zookeeper(start_cluster): node.query("SELECT COUNT() FROM test_table") ## set config to zoo2, server will be normal - new_config = -""" + new_config = """ From a8f325ba84836a01f1d4c3610ec9150a439f15d2 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 3 Nov 2020 14:41:46 +0300 Subject: [PATCH 131/314] Remove redundant check --- src/Interpreters/executeQuery.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index a672b58633d..5a249c15799 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -461,9 +461,6 @@ static std::tuple executeQueryImpl( QueryPipeline & pipeline = res.pipeline; bool use_processors = pipeline.initialized(); - if (res.pipeline.initialized()) - use_processors = true; - if (const auto * insert_interpreter = typeid_cast(&*interpreter)) { /// Save insertion table (not table function). TODO: support remote() table function. From cc8932e7a10fedabca306fe0b0a32abd0f00474d Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 3 Nov 2020 15:51:26 +0300 Subject: [PATCH 132/314] Update DDLWorker.h --- src/Interpreters/DDLWorker.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index 39cdcab709e..75d0ce3f5a4 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -105,7 +105,7 @@ private: private: std::atomic is_circular_replicated = false; - Context & context; + Context context; Poco::Logger * log; std::string host_fqdn; /// current host domain name From bf1ea39392a09e40d6d76a079bffff3ff003894e Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Tue, 3 Nov 2020 16:59:17 +0300 Subject: [PATCH 133/314] fix logs --- src/Interpreters/MetricLog.h | 2 +- src/Interpreters/QueryLog.h | 4 ++-- src/Interpreters/QueryThreadLog.h | 4 ++-- src/Interpreters/TextLog.h | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/MetricLog.h b/src/Interpreters/MetricLog.h index f52d078bdc9..f03b682c60a 100644 --- a/src/Interpreters/MetricLog.h +++ b/src/Interpreters/MetricLog.h @@ -18,7 +18,7 @@ namespace DB struct MetricLogElement { time_t event_time{}; - UInt64 event_time_microseconds{}; + Decimal64 event_time_microseconds{}; UInt64 milliseconds{}; std::vector profile_events; diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 9d42b787160..4b50dc3f57e 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -30,9 +30,9 @@ struct QueryLogElement /// Depending on the type of query and type of stage, not all the fields may be filled. time_t event_time{}; - UInt64 event_time_microseconds{}; + Decimal64 event_time_microseconds{}; time_t query_start_time{}; - UInt64 query_start_time_microseconds{}; + Decimal64 query_start_time_microseconds{}; UInt64 query_duration_ms{}; /// The data fetched from DB to execute the query diff --git a/src/Interpreters/QueryThreadLog.h b/src/Interpreters/QueryThreadLog.h index 5080bfe6919..7a4aa587057 100644 --- a/src/Interpreters/QueryThreadLog.h +++ b/src/Interpreters/QueryThreadLog.h @@ -16,11 +16,11 @@ namespace DB struct QueryThreadLogElement { time_t event_time{}; - UInt64 event_time_microseconds{}; + Decimal64 event_time_microseconds{}; /// When query was attached to current thread time_t query_start_time{}; /// same as above but adds microsecond precision - UInt64 query_start_time_microseconds{}; + Decimal64 query_start_time_microseconds{}; /// Real time spent by the thread to execute the query UInt64 query_duration_ms{}; diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index 814b3c73044..da678868be3 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -9,7 +9,7 @@ using Poco::Message; struct TextLogElement { time_t event_time{}; - UInt64 event_time_microseconds{}; + Decimal64 event_time_microseconds{}; UInt32 microseconds; String thread_name; From 2892252b3695080e73e9326ff505e170c21b2386 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 22 Oct 2020 23:23:44 +0300 Subject: [PATCH 134/314] Support collate in LowCardinality(String) and Nullable(String) and refactor ColumnString --- src/Columns/ColumnLowCardinality.cpp | 78 ++++-- src/Columns/ColumnLowCardinality.h | 9 + src/Columns/ColumnNullable.cpp | 51 +++- src/Columns/ColumnNullable.h | 9 + src/Columns/ColumnString.cpp | 230 +++++------------- src/Columns/ColumnString.h | 16 +- src/Interpreters/sortBlock.cpp | 57 ++++- ...01532_collate_in_low_cardinality.reference | 28 +++ .../01532_collate_in_low_cardinality.sql | 18 ++ .../01533_collate_in_nullable.reference | 36 +++ .../0_stateless/01533_collate_in_nullable.sql | 18 ++ 11 files changed, 342 insertions(+), 208 deletions(-) create mode 100644 tests/queries/0_stateless/01532_collate_in_low_cardinality.reference create mode 100644 tests/queries/0_stateless/01532_collate_in_low_cardinality.sql create mode 100644 tests/queries/0_stateless/01533_collate_in_nullable.reference create mode 100644 tests/queries/0_stateless/01533_collate_in_nullable.sql diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 64b503ed325..2e941a3ef8a 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -14,6 +15,7 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int LOGICAL_ERROR; + extern const int BAD_COLLATION; } namespace @@ -295,14 +297,24 @@ void ColumnLowCardinality::compareColumn(const IColumn & rhs, size_t rhs_row_num compare_results, direction, nan_direction_hint); } -void ColumnLowCardinality::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +void ColumnLowCardinality::getPermutationImpl(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, const Collator * collator) const { if (limit == 0) limit = size(); size_t unique_limit = getDictionary().size(); Permutation unique_perm; - getDictionary().getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm); + if (collator) + { + /// Collations are supported only for ColumnString + const ColumnString * column_string = checkAndGetColumn(getDictionary().getNestedColumn().get()); + if (!column_string) + throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); + + column_string->getPermutationWithCollation(*collator, reverse, unique_limit, unique_perm); + } + else + getDictionary().getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm); /// TODO: optimize with sse. @@ -330,7 +342,8 @@ void ColumnLowCardinality::getPermutation(bool reverse, size_t limit, int nan_di } } -void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const +template +void ColumnLowCardinality::updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const { if (equal_ranges.empty()) return; @@ -345,20 +358,17 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan EqualRanges new_ranges; SCOPE_EXIT({equal_ranges = std::move(new_ranges);}); + auto less = [&comparator](size_t lhs, size_t rhs){ return comparator(lhs, rhs) < 0; }; + for (size_t i = 0; i < number_of_ranges; ++i) { const auto& [first, last] = equal_ranges[i]; - if (reverse) - std::sort(res.begin() + first, res.begin() + last, [this, nan_direction_hint](size_t a, size_t b) - {return getDictionary().compareAt(getIndexes().getUInt(a), getIndexes().getUInt(b), getDictionary(), nan_direction_hint) > 0; }); - else - std::sort(res.begin() + first, res.begin() + last, [this, nan_direction_hint](size_t a, size_t b) - {return getDictionary().compareAt(getIndexes().getUInt(a), getIndexes().getUInt(b), getDictionary(), nan_direction_hint) < 0; }); + std::sort(res.begin() + first, res.begin() + last, less); auto new_first = first; for (auto j = first + 1; j < last; ++j) { - if (compareAt(res[new_first], res[j], *this, nan_direction_hint) != 0) + if (comparator(res[new_first], res[j]) != 0) { if (j - new_first > 1) new_ranges.emplace_back(new_first, j); @@ -379,17 +389,12 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan /// Since then we are working inside the interval. - if (reverse) - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this, nan_direction_hint](size_t a, size_t b) - {return getDictionary().compareAt(getIndexes().getUInt(a), getIndexes().getUInt(b), getDictionary(), nan_direction_hint) > 0; }); - else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this, nan_direction_hint](size_t a, size_t b) - {return getDictionary().compareAt(getIndexes().getUInt(a), getIndexes().getUInt(b), getDictionary(), nan_direction_hint) < 0; }); + std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); auto new_first = first; for (auto j = first + 1; j < limit; ++j) { - if (getDictionary().compareAt(getIndexes().getUInt(res[new_first]), getIndexes().getUInt(res[j]), getDictionary(), nan_direction_hint) != 0) + if (comparator(res[new_first],res[j]) != 0) { if (j - new_first > 1) new_ranges.emplace_back(new_first, j); @@ -401,7 +406,7 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan auto new_last = limit; for (auto j = limit; j < last; ++j) { - if (getDictionary().compareAt(getIndexes().getUInt(res[new_first]), getIndexes().getUInt(res[j]), getDictionary(), nan_direction_hint) == 0) + if (comparator(res[new_first], res[j]) == 0) { std::swap(res[new_last], res[j]); ++new_last; @@ -412,6 +417,43 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan } } +void ColumnLowCardinality::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +{ + getPermutationImpl(reverse, limit, nan_direction_hint, res); +} + +void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const +{ + auto comparator = [this, nan_direction_hint, reverse](size_t lhs, size_t rhs) + { + int ret = getDictionary().compareAt(getIndexes().getUInt(lhs), getIndexes().getUInt(rhs), getDictionary(), nan_direction_hint); + return reverse ? -ret : ret; + }; + + updatePermutationImpl(limit, res, equal_ranges, comparator); +} + +void ColumnLowCardinality::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +{ + getPermutationImpl(reverse, limit, nan_direction_hint, res, &collator); +} + +void ColumnLowCardinality::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const +{ + /// Collations are supported only for ColumnString + const ColumnString * column_string = checkAndGetColumn(getDictionary().getNestedColumn().get()); + if (!column_string) + throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); + + auto comparator = [this, &column_string, &collator, reverse](size_t lhs, size_t rhs) + { + int ret = column_string->compareAtWithCollation(getIndexes().getUInt(lhs), getIndexes().getUInt(rhs), *column_string, collator); + return reverse ? -ret : ret; + }; + + updatePermutationImpl(limit, res, equal_ranges, comparator); +} + std::vector ColumnLowCardinality::scatter(ColumnIndex num_columns, const Selector & selector) const { auto columns = getIndexes().scatter(num_columns, selector); diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 0aeda4567fd..e45449873fc 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -31,6 +31,11 @@ class ColumnLowCardinality final : public COWHelper + void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const; + public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. @@ -129,6 +134,10 @@ public: void updatePermutation(bool reverse, size_t limit, int, IColumn::Permutation & res, EqualRanges & equal_range) const override; + void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const; + + void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges& equal_range) const; + ColumnPtr replicate(const Offsets & offsets) const override { return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().replicate(offsets)); diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 51248a598af..63b86f38342 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include @@ -17,6 +18,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int ILLEGAL_COLUMN; extern const int SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT; + extern const int BAD_COLLATION; } @@ -256,10 +258,21 @@ void ColumnNullable::compareColumn(const IColumn & rhs, size_t rhs_row_num, compare_results, direction, nan_direction_hint); } -void ColumnNullable::getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const +void ColumnNullable::getPermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, const Collator * collator) const { /// Cannot pass limit because of unknown amount of NULLs. - getNestedColumn().getPermutation(reverse, 0, null_direction_hint, res); + + if (collator) + { + /// Collations are supported only for ColumnString + const ColumnString * column_string = checkAndGetColumn(&getNestedColumn()); + if (!column_string) + throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); + + column_string->getPermutationWithCollation(*collator, reverse, 0, res); + } + else + getNestedColumn().getPermutation(reverse, 0, null_direction_hint, res); if ((null_direction_hint > 0) != reverse) { @@ -329,7 +342,7 @@ void ColumnNullable::getPermutation(bool reverse, size_t limit, int null_directi } } -void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const +void ColumnNullable::updatePermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_ranges, const Collator * collator) const { if (equal_ranges.empty()) return; @@ -432,12 +445,42 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire } } - getNestedColumn().updatePermutation(reverse, limit, null_direction_hint, res, new_ranges); + if (collator) + { + /// Collations are supported only for ColumnString + const ColumnString * column_string = checkAndGetColumn(getNestedColumn()); + if (!column_string) + throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); + + column_string->updatePermutationWithCollation(*collator, reverse, limit, null_direction_hint, res, new_ranges); + } + else + getNestedColumn().updatePermutation(reverse, limit, null_direction_hint, res, new_ranges); equal_ranges = std::move(new_ranges); std::move(null_ranges.begin(), null_ranges.end(), std::back_inserter(equal_ranges)); } +void ColumnNullable::getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const +{ + getPermutationImpl(reverse, limit, null_direction_hint, res); +} + +void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const +{ + updatePermutationImpl(reverse, limit, null_direction_hint, res, equal_ranges); +} + +void ColumnNullable::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res) const +{ + getPermutationImpl(reverse, limit, null_direction_hint, res, &collator); +} + +void ColumnNullable::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_range) const +{ + updatePermutationImpl(reverse, limit, null_direction_hint, res, equal_range, &collator); +} + void ColumnNullable::gather(ColumnGathererStream & gatherer) { gatherer.gather(*this); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index e4033e22737..3d7a7970bd3 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -6,6 +6,7 @@ #include #include +class Collator; namespace DB { @@ -30,6 +31,11 @@ private: ColumnNullable(MutableColumnPtr && nested_column_, MutableColumnPtr && null_map_); ColumnNullable(const ColumnNullable &) = default; + void getPermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, const Collator * collator = nullptr) const; + + void updatePermutationImpl( + bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_ranges, const Collator * collator = nullptr) const; + public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. @@ -94,6 +100,9 @@ public: int direction, int nan_direction_hint) const override; void getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_range) const override; + void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res) const; + void updatePermutationWithCollation( + const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_range) const; void reserve(size_t n) override; size_t byteSize() const override; size_t allocatedBytes() const override; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index cd06ea20f83..27dd9e54685 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -284,22 +284,23 @@ void ColumnString::compareColumn( compare_results, direction, nan_direction_hint); } -template -struct ColumnString::less +struct ColumnString::cmp { const ColumnString & parent; - explicit less(const ColumnString & parent_) : parent(parent_) {} - bool operator()(size_t lhs, size_t rhs) const + bool reverse; + cmp(const ColumnString & parent_, bool reverse_=false) : parent(parent_), reverse(reverse_) {} + int operator()(size_t lhs, size_t rhs) const { int res = memcmpSmallAllowOverflow15( parent.chars.data() + parent.offsetAt(lhs), parent.sizeAt(lhs) - 1, parent.chars.data() + parent.offsetAt(rhs), parent.sizeAt(rhs) - 1); - return positive ? (res < 0) : (res > 0); + return reverse ? -res : res; } }; -void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res) const +template +void ColumnString::getPermutationImpl(size_t limit, Permutation & res, Cmp comparator) const { size_t s = offsets.size(); res.resize(s); @@ -309,23 +310,16 @@ void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_directio if (limit >= s) limit = 0; + auto less = [&comparator](size_t lhs, size_t rhs){ return comparator(lhs, rhs) < 0; }; + if (limit) - { - if (reverse) - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); - else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); - } + std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); else - { - if (reverse) - std::sort(res.begin(), res.end(), less(*this)); - else - std::sort(res.begin(), res.end(), less(*this)); - } + std::sort(res.begin(), res.end(), less); } -void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res, EqualRanges & equal_ranges) const +template +void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const { if (equal_ranges.empty()) return; @@ -340,21 +334,17 @@ void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direc if (limit) --number_of_ranges; + auto less = [&comparator](size_t lhs, size_t rhs){ return comparator(lhs, rhs) < 0; }; + for (size_t i = 0; i < number_of_ranges; ++i) { const auto & [first, last] = equal_ranges[i]; - - if (reverse) - std::sort(res.begin() + first, res.begin() + last, less(*this)); - else - std::sort(res.begin() + first, res.begin() + last, less(*this)); + std::sort(res.begin() + first, res.begin() + last, less); size_t new_first = first; for (size_t j = first + 1; j < last; ++j) { - if (memcmpSmallAllowOverflow15( - chars.data() + offsetAt(res[j]), sizeAt(res[j]) - 1, - chars.data() + offsetAt(res[new_first]), sizeAt(res[new_first]) - 1) != 0) + if (comparator(res[j], res[new_first]) != 0) { if (j - new_first > 1) new_ranges.emplace_back(new_first, j); @@ -375,17 +365,12 @@ void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direc /// Since then we are working inside the interval. - if (reverse) - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); - else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); + std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); size_t new_first = first; for (size_t j = first + 1; j < limit; ++j) { - if (memcmpSmallAllowOverflow15( - chars.data() + offsetAt(res[j]), sizeAt(res[j]) - 1, - chars.data() + offsetAt(res[new_first]), sizeAt(res[new_first]) - 1) != 0) + if (comparator(res[j], res[new_first]) != 0) { if (j - new_first > 1) new_ranges.emplace_back(new_first, j); @@ -395,9 +380,7 @@ void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direc size_t new_last = limit; for (size_t j = limit; j < last; ++j) { - if (memcmpSmallAllowOverflow15( - chars.data() + offsetAt(res[j]), sizeAt(res[j]) - 1, - chars.data() + offsetAt(res[new_first]), sizeAt(res[new_first]) - 1) == 0) + if (comparator(res[j], res[new_first]) == 0) { std::swap(res[j], res[new_last]); ++new_last; @@ -408,6 +391,45 @@ void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direc } } +void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res) const +{ + getPermutationImpl(limit, res, cmp(*this, reverse)); +} + +void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res, EqualRanges & equal_ranges) const +{ + updatePermutationImpl(limit, res, equal_ranges, cmp(*this, reverse)); +} + +struct ColumnString::cmpWithCollation +{ + const ColumnString & parent; + const Collator & collator; + bool reverse; + + cmpWithCollation(const ColumnString & parent_, const Collator & collator_, bool reverse_=false) : parent(parent_), collator(collator_), reverse(reverse_) {} + + int operator()(size_t lhs, size_t rhs) const + { + int res = collator.compare( + reinterpret_cast(&parent.chars[parent.offsetAt(lhs)]), parent.sizeAt(lhs), + reinterpret_cast(&parent.chars[parent.offsetAt(rhs)]), parent.sizeAt(rhs)); + + return reverse ? -res : res; + } +}; + +void ColumnString::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, Permutation & res) const +{ + getPermutationImpl(limit, res, cmpWithCollation(*this, collator, reverse)); +} + +void ColumnString::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const +{ + updatePermutationImpl(limit, res, equal_ranges, cmpWithCollation(*this, collator, reverse)); +} + + ColumnPtr ColumnString::replicate(const Offsets & replicate_offsets) const { size_t col_size = size(); @@ -476,13 +498,13 @@ void ColumnString::getExtremes(Field & min, Field & max) const size_t min_idx = 0; size_t max_idx = 0; - less less_op(*this); + cmp cmp_op(*this); for (size_t i = 1; i < col_size; ++i) { - if (less_op(i, min_idx)) + if (cmp_op(i, min_idx) < 0) min_idx = i; - else if (less_op(max_idx, i)) + else if (cmp_op(max_idx, i) < 0) max_idx = i; } @@ -500,134 +522,6 @@ int ColumnString::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs reinterpret_cast(&rhs.chars[rhs.offsetAt(m)]), rhs.sizeAt(m)); } - -template -struct ColumnString::lessWithCollation -{ - const ColumnString & parent; - const Collator & collator; - - lessWithCollation(const ColumnString & parent_, const Collator & collator_) : parent(parent_), collator(collator_) {} - - bool operator()(size_t lhs, size_t rhs) const - { - int res = collator.compare( - reinterpret_cast(&parent.chars[parent.offsetAt(lhs)]), parent.sizeAt(lhs), - reinterpret_cast(&parent.chars[parent.offsetAt(rhs)]), parent.sizeAt(rhs)); - - return positive ? (res < 0) : (res > 0); - } -}; - -void ColumnString::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, Permutation & res) const -{ - size_t s = offsets.size(); - res.resize(s); - for (size_t i = 0; i < s; ++i) - res[i] = i; - - if (limit >= s) - limit = 0; - - if (limit) - { - if (reverse) - std::partial_sort(res.begin(), res.begin() + limit, res.end(), lessWithCollation(*this, collator)); - else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), lessWithCollation(*this, collator)); - } - else - { - if (reverse) - std::sort(res.begin(), res.end(), lessWithCollation(*this, collator)); - else - std::sort(res.begin(), res.end(), lessWithCollation(*this, collator)); - } -} - -void ColumnString::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const -{ - if (equal_ranges.empty()) - return; - - if (limit >= size() || limit >= equal_ranges.back().second) - limit = 0; - - size_t number_of_ranges = equal_ranges.size(); - if (limit) - --number_of_ranges; - - EqualRanges new_ranges; - SCOPE_EXIT({equal_ranges = std::move(new_ranges);}); - - for (size_t i = 0; i < number_of_ranges; ++i) - { - const auto& [first, last] = equal_ranges[i]; - - if (reverse) - std::sort(res.begin() + first, res.begin() + last, lessWithCollation(*this, collator)); - else - std::sort(res.begin() + first, res.begin() + last, lessWithCollation(*this, collator)); - auto new_first = first; - for (auto j = first + 1; j < last; ++j) - { - if (collator.compare( - reinterpret_cast(&chars[offsetAt(res[new_first])]), sizeAt(res[new_first]), - reinterpret_cast(&chars[offsetAt(res[j])]), sizeAt(res[j])) != 0) - { - if (j - new_first > 1) - new_ranges.emplace_back(new_first, j); - - new_first = j; - } - } - if (last - new_first > 1) - new_ranges.emplace_back(new_first, last); - } - - if (limit) - { - const auto & [first, last] = equal_ranges.back(); - - if (limit < first || limit > last) - return; - - /// Since then we are working inside the interval. - - if (reverse) - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, lessWithCollation(*this, collator)); - else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, lessWithCollation(*this, collator)); - - auto new_first = first; - for (auto j = first + 1; j < limit; ++j) - { - if (collator.compare( - reinterpret_cast(&chars[offsetAt(res[new_first])]), sizeAt(res[new_first]), - reinterpret_cast(&chars[offsetAt(res[j])]), sizeAt(res[j])) != 0) - { - if (j - new_first > 1) - new_ranges.emplace_back(new_first, j); - - new_first = j; - } - } - auto new_last = limit; - for (auto j = limit; j < last; ++j) - { - if (collator.compare( - reinterpret_cast(&chars[offsetAt(res[new_first])]), sizeAt(res[new_first]), - reinterpret_cast(&chars[offsetAt(res[j])]), sizeAt(res[j])) == 0) - { - std::swap(res[new_last], res[j]); - ++new_last; - } - } - if (new_last - new_first > 1) - new_ranges.emplace_back(new_first, new_last); - } -} - void ColumnString::protect() { getChars().protect(); diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 19398e07b83..c91d982f126 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -42,15 +42,19 @@ private: /// Size of i-th element, including terminating zero. size_t ALWAYS_INLINE sizeAt(ssize_t i) const { return offsets[i] - offsets[i - 1]; } - template - struct less; + struct cmp; - template - struct lessWithCollation; + struct cmpWithCollation; ColumnString() = default; ColumnString(const ColumnString & src); + template + void getPermutationImpl(size_t limit, Permutation & res, Cmp comparator) const; + + template + void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const; + public: const char * getFamilyName() const override { return "String"; } TypeIndex getDataType() const override { return TypeIndex::String; } @@ -233,12 +237,12 @@ public: void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; - void updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_range) const override; + void updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const override; /// Sorting with respect of collation. void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, Permutation & res) const; - void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges& equal_range) const; + void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const; ColumnPtr replicate(const Offsets & replicate_offsets) const override; diff --git a/src/Interpreters/sortBlock.cpp b/src/Interpreters/sortBlock.cpp index c2436806fcd..5d114c746e5 100644 --- a/src/Interpreters/sortBlock.cpp +++ b/src/Interpreters/sortBlock.cpp @@ -2,6 +2,8 @@ #include #include +#include +#include #include #include @@ -20,6 +22,24 @@ static bool isCollationRequired(const SortColumnDescription & description) return description.collator != nullptr; } +static bool isCollationSupported(const IColumn * column) +{ + if (column->getDataType() == TypeIndex::String) + return true; + + if (column->getDataType() == TypeIndex::Nullable) + { + const ColumnNullable * column_nullable = assert_cast(column); + return isCollationSupported(&column_nullable->getNestedColumn()); + } + + if (column->getDataType() == TypeIndex::LowCardinality) + { + const ColumnLowCardinality * column_low_cardinality = assert_cast(column); + return isCollationSupported(column_low_cardinality->getDictionary().getNestedColumn().get()); + } + return false; +} ColumnsWithSortDescriptions getColumnsWithSortDescription(const Block & block, const SortDescription & description) { @@ -101,7 +121,6 @@ struct PartialSortingLessWithCollation } }; - void sortBlock(Block & block, const SortDescription & description, UInt64 limit) { if (!block) @@ -120,14 +139,18 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) bool is_column_const = false; if (isCollationRequired(description[0])) { - /// it it's real string column, than we need sort + /// Check if column supports collations + if (!isCollationSupported(column)) + throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); + if (const ColumnString * column_string = checkAndGetColumn(column)) column_string->getPermutationWithCollation(*description[0].collator, reverse, limit, perm); - else if (checkAndGetColumnConstData(column)) + else if (const ColumnNullable * column_nullable = checkAndGetColumn(column)) + column_nullable->getPermutationWithCollation(*description[0].collator, reverse, limit, description[0].nulls_direction, perm); + else if (const ColumnLowCardinality * column_low_cardinality = checkAndGetColumn(column)) + column_low_cardinality->getPermutationWithCollation(*description[0].collator, reverse, limit, description[0].nulls_direction, perm); + else if (isColumnConst(*column)) is_column_const = true; - else - throw Exception("Collations could be specified only for String columns.", ErrorCodes::BAD_COLLATION); - } else if (!isColumnConst(*column)) { @@ -163,8 +186,8 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) const IColumn * column = columns_with_sort_desc[i].column; if (isCollationRequired(description[i])) { - if (!checkAndGetColumn(column) && !checkAndGetColumnConstData(column)) - throw Exception("Collations could be specified only for String columns.", ErrorCodes::BAD_COLLATION); + if (!isCollationSupported(column)) + throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); need_collation = true; } @@ -187,10 +210,20 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) if (isCollationRequired(column.description)) { - const ColumnString & column_string = assert_cast(*column.column); - column_string.updatePermutationWithCollation( - *column.description.collator, - column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); + if (const ColumnString * column_string = checkAndGetColumn(column.column)) + column_string->updatePermutationWithCollation( + *column.description.collator, + column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); + + else if (const ColumnNullable * column_nullable = checkAndGetColumn(column.column)) + column_nullable->updatePermutationWithCollation( + *column.description.collator, + column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); + + else if (const ColumnLowCardinality * column_low_cardinality = checkAndGetColumn(column.column)) + column_low_cardinality->updatePermutationWithCollation( + *column.description.collator, + column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); } else { diff --git a/tests/queries/0_stateless/01532_collate_in_low_cardinality.reference b/tests/queries/0_stateless/01532_collate_in_low_cardinality.reference new file mode 100644 index 00000000000..b7a4830f9cf --- /dev/null +++ b/tests/queries/0_stateless/01532_collate_in_low_cardinality.reference @@ -0,0 +1,28 @@ +Order by without collate +1 Ё +2 А +2 Я +1 а +2 я +1 ё +Order by with collate +1 а +2 А +1 ё +1 Ё +2 я +2 Я +Order by tuple without collate +1 Ё +1 а +1 ё +2 А +2 Я +2 я +Order by tuple with collate +1 а +1 ё +1 Ё +2 А +2 я +2 Я diff --git a/tests/queries/0_stateless/01532_collate_in_low_cardinality.sql b/tests/queries/0_stateless/01532_collate_in_low_cardinality.sql new file mode 100644 index 00000000000..0f4194ee671 --- /dev/null +++ b/tests/queries/0_stateless/01532_collate_in_low_cardinality.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test_collate; + +CREATE TABLE test_collate (x UInt32, s LowCardinality(String)) ENGINE=Memory(); + +INSERT INTO test_collate VALUES (1, 'Ё'), (1, 'ё'), (1, 'а'), (2, 'А'), (2, 'я'), (2, 'Я'); + +SELECT 'Order by without collate'; +SELECT * FROM test_collate ORDER BY s; +SELECT 'Order by with collate'; +SELECT * FROM test_collate ORDER BY s COLLATE 'ru'; + +SELECT 'Order by tuple without collate'; +SELECT * FROM test_collate ORDER BY x, s; +SELECT 'Order by tuple with collate'; +SELECT * FROM test_collate ORDER BY x, s COLLATE 'ru'; + +DROP TABLE test_collate; + diff --git a/tests/queries/0_stateless/01533_collate_in_nullable.reference b/tests/queries/0_stateless/01533_collate_in_nullable.reference new file mode 100644 index 00000000000..6bb06cbc8b5 --- /dev/null +++ b/tests/queries/0_stateless/01533_collate_in_nullable.reference @@ -0,0 +1,36 @@ +Order by without collate +1 Ё +2 А +2 Я +1 а +2 я +1 ё +1 \N +2 \N +Order by with collate +1 а +2 А +1 ё +1 Ё +2 я +2 Я +1 \N +2 \N +Order by tuple without collate +1 Ё +1 а +1 ё +1 \N +2 А +2 Я +2 я +2 \N +Order by tuple with collate +1 а +1 ё +1 Ё +1 \N +2 А +2 я +2 Я +2 \N diff --git a/tests/queries/0_stateless/01533_collate_in_nullable.sql b/tests/queries/0_stateless/01533_collate_in_nullable.sql new file mode 100644 index 00000000000..40b48bee465 --- /dev/null +++ b/tests/queries/0_stateless/01533_collate_in_nullable.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test_collate; + +CREATE TABLE test_collate (x UInt32, s Nullable(String)) ENGINE=Memory(); + +INSERT INTO test_collate VALUES (1, 'Ё'), (1, 'ё'), (1, 'а'), (1, null), (2, 'А'), (2, 'я'), (2, 'Я'), (2, null); + +SELECT 'Order by without collate'; +SELECT * FROM test_collate ORDER BY s; +SELECT 'Order by with collate'; +SELECT * FROM test_collate ORDER BY s COLLATE 'ru'; + +SELECT 'Order by tuple without collate'; +SELECT * FROM test_collate ORDER BY x, s; +SELECT 'Order by tuple with collate'; +SELECT * FROM test_collate ORDER BY x, s COLLATE 'ru'; + +DROP TABLE test_collate; + From 18d8d5d81f5844f77ba3bac972dc7f893fc014d7 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 23 Oct 2020 11:02:40 +0300 Subject: [PATCH 135/314] Add tests in arcadia_skip_list.txt --- tests/queries/0_stateless/arcadia_skip_list.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index f5b81c08520..cc03dee9eb0 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -155,5 +155,9 @@ 01509_dictionary_preallocate 01526_max_untracked_memory 01530_drop_database_atomic_sync +01532_collate_in_low_cardinality +01533_collate_in_nullable +01542_collate_in_array +01543_collate_in_tuple 01547_query_log_current_database 01548_query_log_query_execution_ms From 97a6e3dde2bb0b99a1323e05370ae07d6fc3012c Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sat, 24 Oct 2020 20:15:03 +0300 Subject: [PATCH 136/314] Skip collate test in fasttest --- docker/test/fasttest/run.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 8300c31681e..f11e51dee98 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -240,6 +240,8 @@ TESTS_TO_SKIP=( 01354_order_by_tuple_collate_const 01355_ilike 01411_bayesian_ab_testing + 01532_collate_in_low_cardinality + 01533_collate_in_nullable _orc_ arrow avro From 5c296365e2c808e4ab601c389f38c7485f072e0b Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 27 Oct 2020 14:12:48 +0300 Subject: [PATCH 137/314] Fix build error --- src/Columns/ColumnString.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 27dd9e54685..3093ae10646 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -288,7 +288,7 @@ struct ColumnString::cmp { const ColumnString & parent; bool reverse; - cmp(const ColumnString & parent_, bool reverse_=false) : parent(parent_), reverse(reverse_) {} + explicit cmp(const ColumnString & parent_, bool reverse_=false) : parent(parent_), reverse(reverse_) {} int operator()(size_t lhs, size_t rhs) const { int res = memcmpSmallAllowOverflow15( From 4d399fff3ee8a0922ca3b33e80daf6fae5730d69 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 29 Oct 2020 14:24:01 +0300 Subject: [PATCH 138/314] Support collation for Array and Tuple --- docker/test/fasttest/run.sh | 2 + src/Columns/ColumnArray.cpp | 120 +++++++++++------- src/Columns/ColumnArray.h | 15 +++ src/Columns/ColumnLowCardinality.cpp | 33 ++--- src/Columns/ColumnLowCardinality.h | 19 ++- src/Columns/ColumnNullable.cpp | 30 +++-- src/Columns/ColumnNullable.h | 20 +-- src/Columns/ColumnString.cpp | 41 +++--- src/Columns/ColumnString.h | 20 +-- src/Columns/ColumnTuple.cpp | 89 ++++++++++--- src/Columns/ColumnTuple.h | 16 ++- src/Columns/IColumn.h | 23 +++- src/Core/SortCursor.h | 7 +- src/Interpreters/sortBlock.cpp | 56 ++------ ...01532_collate_in_low_cardinality.reference | 36 ++++++ .../01532_collate_in_low_cardinality.sql | 17 ++- .../01542_collate_in_array.reference | 50 ++++++++ .../0_stateless/01542_collate_in_array.sql | 34 +++++ .../01543_collate_in_tuple.reference | 60 +++++++++ .../0_stateless/01543_collate_in_tuple.sql | 34 +++++ 20 files changed, 524 insertions(+), 198 deletions(-) create mode 100644 tests/queries/0_stateless/01542_collate_in_array.reference create mode 100644 tests/queries/0_stateless/01542_collate_in_array.sql create mode 100644 tests/queries/0_stateless/01543_collate_in_tuple.reference create mode 100644 tests/queries/0_stateless/01543_collate_in_tuple.sql diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index f11e51dee98..ad25be9e2de 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -242,6 +242,8 @@ TESTS_TO_SKIP=( 01411_bayesian_ab_testing 01532_collate_in_low_cardinality 01533_collate_in_nullable + 01542_collate_in_array + 01543_collate_in_tuple _orc_ arrow avro diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index e4d17c586ac..c061dd50642 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -324,8 +324,7 @@ void ColumnArray::popBack(size_t n) offsets_data.resize_assume_reserved(offsets_data.size() - n); } - -int ColumnArray::compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const +int ColumnArray::compareAtImpl(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint, const Collator * collator) const { const ColumnArray & rhs = assert_cast(rhs_); @@ -334,8 +333,15 @@ int ColumnArray::compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_dir size_t rhs_size = rhs.sizeAt(m); size_t min_size = std::min(lhs_size, rhs_size); for (size_t i = 0; i < min_size; ++i) - if (int res = getData().compareAt(offsetAt(n) + i, rhs.offsetAt(m) + i, *rhs.data.get(), nan_direction_hint)) + { + int res; + if (collator) + res = getData().compareAtWithCollation(offsetAt(n) + i, rhs.offsetAt(m) + i, *rhs.data.get(), nan_direction_hint, *collator); + else + res = getData().compareAt(offsetAt(n) + i, rhs.offsetAt(m) + i, *rhs.data.get(), nan_direction_hint); + if (res) return res; + } return lhs_size < rhs_size ? -1 @@ -344,6 +350,16 @@ int ColumnArray::compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_dir : 1); } +int ColumnArray::compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const +{ + return compareAtImpl(n, m, rhs_, nan_direction_hint); +} + +int ColumnArray::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint, const Collator & collator) const +{ + return compareAtImpl(n, m, rhs_, nan_direction_hint, &collator); +} + void ColumnArray::compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const @@ -352,27 +368,25 @@ void ColumnArray::compareColumn(const IColumn & rhs, size_t rhs_row_num, compare_results, direction, nan_direction_hint); } -namespace -{ - template - struct Less +struct ColumnArray::Cmp { + const ColumnArray & parent; + int nan_direction_hint; + bool reverse; + const Collator * collator; + + Cmp(const ColumnArray & parent_, int nan_direction_hint_, bool reverse_=false, const Collator * collator_=nullptr) + : parent(parent_), nan_direction_hint(nan_direction_hint_), reverse(reverse_), collator(collator_) {} + + int operator()(size_t lhs, size_t rhs) const { - const ColumnArray & parent; - int nan_direction_hint; - - Less(const ColumnArray & parent_, int nan_direction_hint_) - : parent(parent_), nan_direction_hint(nan_direction_hint_) {} - - bool operator()(size_t lhs, size_t rhs) const - { - if (positive) - return parent.compareAt(lhs, rhs, parent, nan_direction_hint) < 0; - else - return parent.compareAt(lhs, rhs, parent, nan_direction_hint) > 0; - } - }; -} - + int res; + if (collator) + res = parent.compareAtWithCollation(lhs, rhs, parent, nan_direction_hint, *collator); + else + res = parent.compareAt(lhs, rhs, parent, nan_direction_hint); + return reverse ? -res : res; + } +}; void ColumnArray::reserve(size_t n) { @@ -753,7 +767,8 @@ ColumnPtr ColumnArray::indexImpl(const PaddedPODArray & indexes, size_t limit INSTANTIATE_INDEX_IMPL(ColumnArray) -void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +template +void ColumnArray::getPermutationImpl(size_t limit, Permutation & res, Comparator cmp) const { size_t s = size(); if (limit >= s) @@ -763,23 +778,16 @@ void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_h for (size_t i = 0; i < s; ++i) res[i] = i; + auto less = [&cmp](size_t lhs, size_t rhs){ return cmp(lhs, rhs) < 0; }; + if (limit) - { - if (reverse) - std::partial_sort(res.begin(), res.begin() + limit, res.end(), Less(*this, nan_direction_hint)); - else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), Less(*this, nan_direction_hint)); - } + std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); else - { - if (reverse) - std::sort(res.begin(), res.end(), Less(*this, nan_direction_hint)); - else - std::sort(res.begin(), res.end(), Less(*this, nan_direction_hint)); - } + std::sort(res.begin(), res.end(), less); } -void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_range) const +template +void ColumnArray::updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_range, Comparator cmp) const { if (equal_range.empty()) return; @@ -792,20 +800,19 @@ void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_directio if (limit) --number_of_ranges; + auto less = [&cmp](size_t lhs, size_t rhs){ return cmp(lhs, rhs) < 0; }; + EqualRanges new_ranges; for (size_t i = 0; i < number_of_ranges; ++i) { const auto & [first, last] = equal_range[i]; - if (reverse) - std::sort(res.begin() + first, res.begin() + last, Less(*this, nan_direction_hint)); - else - std::sort(res.begin() + first, res.begin() + last, Less(*this, nan_direction_hint)); + std::sort(res.begin() + first, res.begin() + last, less); auto new_first = first; for (auto j = first + 1; j < last; ++j) { - if (compareAt(res[new_first], res[j], *this, nan_direction_hint) != 0) + if (cmp(res[new_first], res[j]) != 0) { if (j - new_first > 1) new_ranges.emplace_back(new_first, j); @@ -827,14 +834,11 @@ void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_directio /// Since then we are working inside the interval. - if (reverse) - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, Less(*this, nan_direction_hint)); - else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, Less(*this, nan_direction_hint)); + std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); auto new_first = first; for (auto j = first + 1; j < limit; ++j) { - if (compareAt(res[new_first], res[j], *this, nan_direction_hint) != 0) + if (cmp(res[new_first], res[j]) != 0) { if (j - new_first > 1) new_ranges.emplace_back(new_first, j); @@ -845,7 +849,7 @@ void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_directio auto new_last = limit; for (auto j = limit; j < last; ++j) { - if (compareAt(res[new_first], res[j], *this, nan_direction_hint) == 0) + if (cmp(res[new_first], res[j]) == 0) { std::swap(res[new_last], res[j]); ++new_last; @@ -859,6 +863,26 @@ void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_directio equal_range = std::move(new_ranges); } +void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +{ + getPermutationImpl(limit, res, Cmp(*this, nan_direction_hint, reverse)); +} + +void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_range) const +{ + updatePermutationImpl(limit, res, equal_range, Cmp(*this, nan_direction_hint, reverse)); +} + +void ColumnArray::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +{ + getPermutationImpl(limit, res, Cmp(*this, nan_direction_hint, reverse, &collator)); +} + +void ColumnArray::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_range) const +{ + updatePermutationImpl(limit, res, equal_range, Cmp(*this, nan_direction_hint, reverse, &collator)); +} + ColumnPtr ColumnArray::replicate(const Offsets & replicate_offsets) const { if (replicate_offsets.empty()) diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index cec8387ab66..028eaba73c5 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -77,8 +77,11 @@ public: void compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; + int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint, const Collator & collator) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_range) const override; + void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; + void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges& equal_range) const override; void reserve(size_t n) override; size_t byteSize() const override; size_t allocatedBytes() const override; @@ -132,6 +135,8 @@ public: return false; } + bool isCollationSupported() const override { return getData().isCollationSupported(); } + private: WrappedPtr data; WrappedPtr offsets; @@ -169,6 +174,16 @@ private: ColumnPtr filterTuple(const Filter & filt, ssize_t result_size_hint) const; ColumnPtr filterNullable(const Filter & filt, ssize_t result_size_hint) const; ColumnPtr filterGeneric(const Filter & filt, ssize_t result_size_hint) const; + + int compareAtImpl(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint, const Collator * collator=nullptr) const; + + template + void getPermutationImpl(size_t limit, Permutation & res, Comparator cmp) const; + + template + void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_range, Comparator cmp) const; + + struct Cmp; }; diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 2e941a3ef8a..37e97da88b9 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -15,7 +15,6 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int LOGICAL_ERROR; - extern const int BAD_COLLATION; } namespace @@ -280,14 +279,26 @@ MutableColumnPtr ColumnLowCardinality::cloneResized(size_t size) const return ColumnLowCardinality::create(IColumn::mutate(std::move(unique_ptr)), getIndexes().cloneResized(size)); } -int ColumnLowCardinality::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +int ColumnLowCardinality::compareAtImpl(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator * collator) const { const auto & low_cardinality_column = assert_cast(rhs); size_t n_index = getIndexes().getUInt(n); size_t m_index = low_cardinality_column.getIndexes().getUInt(m); + if (collator) + return getDictionary().getNestedColumn()->compareAtWithCollation(n_index, m_index, *low_cardinality_column.getDictionary().getNestedColumn(), nan_direction_hint, *collator); return getDictionary().compareAt(n_index, m_index, low_cardinality_column.getDictionary(), nan_direction_hint); } +int ColumnLowCardinality::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +{ + return compareAtImpl(n, m, rhs, nan_direction_hint); +} + +int ColumnLowCardinality::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator & collator) const +{ + return compareAtImpl(n, m, rhs, nan_direction_hint, &collator); +} + void ColumnLowCardinality::compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const @@ -306,12 +317,7 @@ void ColumnLowCardinality::getPermutationImpl(bool reverse, size_t limit, int na Permutation unique_perm; if (collator) { - /// Collations are supported only for ColumnString - const ColumnString * column_string = checkAndGetColumn(getDictionary().getNestedColumn().get()); - if (!column_string) - throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); - - column_string->getPermutationWithCollation(*collator, reverse, unique_limit, unique_perm); + getDictionary().getNestedColumn()->getPermutationWithCollation(*collator, reverse, unique_limit, nan_direction_hint, unique_perm); } else getDictionary().getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm); @@ -438,16 +444,11 @@ void ColumnLowCardinality::getPermutationWithCollation(const Collator & collator getPermutationImpl(reverse, limit, nan_direction_hint, res, &collator); } -void ColumnLowCardinality::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const +void ColumnLowCardinality::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_ranges) const { - /// Collations are supported only for ColumnString - const ColumnString * column_string = checkAndGetColumn(getDictionary().getNestedColumn().get()); - if (!column_string) - throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); - - auto comparator = [this, &column_string, &collator, reverse](size_t lhs, size_t rhs) + auto comparator = [this, &collator, reverse, nan_direction_hint](size_t lhs, size_t rhs) { - int ret = column_string->compareAtWithCollation(getIndexes().getUInt(lhs), getIndexes().getUInt(rhs), *column_string, collator); + int ret = getDictionary().getNestedColumn()->compareAtWithCollation(getIndexes().getUInt(lhs), getIndexes().getUInt(rhs), *getDictionary().getNestedColumn(), nan_direction_hint, collator); return reverse ? -ret : ret; }; diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index e45449873fc..0874f619b8a 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -31,11 +31,6 @@ class ColumnLowCardinality final : public COWHelper - void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const; - public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. @@ -130,13 +125,15 @@ public: PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; + int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator &) const override; + void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int, IColumn::Permutation & res, EqualRanges & equal_range) const override; - void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const; + void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; - void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges& equal_range) const; + void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges& equal_range) const override; ColumnPtr replicate(const Offsets & offsets) const override { @@ -179,6 +176,7 @@ public: size_t sizeOfValueIfFixed() const override { return getDictionary().sizeOfValueIfFixed(); } bool isNumeric() const override { return getDictionary().isNumeric(); } bool lowCardinality() const override { return true; } + bool isCollationSupported() const override { return getDictionary().getNestedColumn()->isCollationSupported(); } /** * Checks if the dictionary column is Nullable(T). @@ -318,6 +316,13 @@ private: void compactInplace(); void compactIfSharedDictionary(); + + int compareAtImpl(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator * collator=nullptr) const; + + void getPermutationImpl(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, const Collator * collator = nullptr) const; + + template + void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const; }; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 63b86f38342..cbb82264694 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -18,7 +18,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int ILLEGAL_COLUMN; extern const int SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT; - extern const int BAD_COLLATION; } @@ -225,7 +224,7 @@ ColumnPtr ColumnNullable::index(const IColumn & indexes, size_t limit) const return ColumnNullable::create(indexed_data, indexed_null_map); } -int ColumnNullable::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const +int ColumnNullable::compareAtImpl(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint, const Collator * collator) const { /// NULL values share the properties of NaN values. /// Here the last parameter of compareAt is called null_direction_hint @@ -247,9 +246,22 @@ int ColumnNullable::compareAt(size_t n, size_t m, const IColumn & rhs_, int null } const IColumn & nested_rhs = nullable_rhs.getNestedColumn(); + if (collator) + return getNestedColumn().compareAtWithCollation(n, m, nested_rhs, null_direction_hint, *collator); + return getNestedColumn().compareAt(n, m, nested_rhs, null_direction_hint); } +int ColumnNullable::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const +{ + return compareAtImpl(n, m, rhs_, null_direction_hint); +} + +int ColumnNullable::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint, const Collator & collator) const +{ + return compareAtImpl(n, m, rhs_, null_direction_hint, &collator); +} + void ColumnNullable::compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const @@ -264,12 +276,7 @@ void ColumnNullable::getPermutationImpl(bool reverse, size_t limit, int null_dir if (collator) { - /// Collations are supported only for ColumnString - const ColumnString * column_string = checkAndGetColumn(&getNestedColumn()); - if (!column_string) - throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); - - column_string->getPermutationWithCollation(*collator, reverse, 0, res); + getNestedColumn().getPermutationWithCollation(*collator, reverse, 0, null_direction_hint, res); } else getNestedColumn().getPermutation(reverse, 0, null_direction_hint, res); @@ -447,12 +454,7 @@ void ColumnNullable::updatePermutationImpl(bool reverse, size_t limit, int null_ if (collator) { - /// Collations are supported only for ColumnString - const ColumnString * column_string = checkAndGetColumn(getNestedColumn()); - if (!column_string) - throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); - - column_string->updatePermutationWithCollation(*collator, reverse, limit, null_direction_hint, res, new_ranges); + getNestedColumn().updatePermutationWithCollation(*collator, reverse, limit, null_direction_hint, res, new_ranges); } else getNestedColumn().updatePermutation(reverse, limit, null_direction_hint, res, new_ranges); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 3d7a7970bd3..47b0103eab4 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -31,11 +31,6 @@ private: ColumnNullable(MutableColumnPtr && nested_column_, MutableColumnPtr && null_map_); ColumnNullable(const ColumnNullable &) = default; - void getPermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, const Collator * collator = nullptr) const; - - void updatePermutationImpl( - bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_ranges, const Collator * collator = nullptr) const; - public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. @@ -98,11 +93,12 @@ public: void compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; + int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int null_direction_hint, const Collator &) const override; void getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override; - void updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_range) const override; - void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res) const; + void updatePermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_range) const override; + void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override; void updatePermutationWithCollation( - const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_range) const; + const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_range) const override; void reserve(size_t n) override; size_t byteSize() const override; size_t allocatedBytes() const override; @@ -138,6 +134,7 @@ public: bool valuesHaveFixedSize() const override { return nested_column->valuesHaveFixedSize(); } size_t sizeOfValueIfFixed() const override { return null_map->sizeOfValueIfFixed() + nested_column->sizeOfValueIfFixed(); } bool onlyNull() const override { return nested_column->isDummy(); } + bool isCollationSupported() const override { return nested_column->isCollationSupported(); } /// Return the column that represents values. @@ -173,6 +170,13 @@ private: template void applyNullMapImpl(const ColumnUInt8 & map); + + int compareAtImpl(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint, const Collator * collator=nullptr) const; + + void getPermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, const Collator * collator = nullptr) const; + + void updatePermutationImpl( + bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_ranges, const Collator * collator = nullptr) const; }; ColumnPtr makeNullable(const ColumnPtr & column); diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 3093ae10646..9ea12041d85 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -284,11 +284,11 @@ void ColumnString::compareColumn( compare_results, direction, nan_direction_hint); } -struct ColumnString::cmp +struct ColumnString::Cmp { const ColumnString & parent; bool reverse; - explicit cmp(const ColumnString & parent_, bool reverse_=false) : parent(parent_), reverse(reverse_) {} + explicit Cmp(const ColumnString & parent_, bool reverse_=false) : parent(parent_), reverse(reverse_) {} int operator()(size_t lhs, size_t rhs) const { int res = memcmpSmallAllowOverflow15( @@ -299,8 +299,8 @@ struct ColumnString::cmp } }; -template -void ColumnString::getPermutationImpl(size_t limit, Permutation & res, Cmp comparator) const +template +void ColumnString::getPermutationImpl(size_t limit, Permutation & res, Comparator cmp) const { size_t s = offsets.size(); res.resize(s); @@ -310,7 +310,7 @@ void ColumnString::getPermutationImpl(size_t limit, Permutation & res, Cmp compa if (limit >= s) limit = 0; - auto less = [&comparator](size_t lhs, size_t rhs){ return comparator(lhs, rhs) < 0; }; + auto less = [&cmp](size_t lhs, size_t rhs){ return cmp(lhs, rhs) < 0; }; if (limit) std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); @@ -318,8 +318,8 @@ void ColumnString::getPermutationImpl(size_t limit, Permutation & res, Cmp compa std::sort(res.begin(), res.end(), less); } -template -void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const +template +void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Comparator cmp) const { if (equal_ranges.empty()) return; @@ -334,7 +334,7 @@ void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualR if (limit) --number_of_ranges; - auto less = [&comparator](size_t lhs, size_t rhs){ return comparator(lhs, rhs) < 0; }; + auto less = [&cmp](size_t lhs, size_t rhs){ return cmp(lhs, rhs) < 0; }; for (size_t i = 0; i < number_of_ranges; ++i) { @@ -344,7 +344,7 @@ void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualR size_t new_first = first; for (size_t j = first + 1; j < last; ++j) { - if (comparator(res[j], res[new_first]) != 0) + if (cmp(res[j], res[new_first]) != 0) { if (j - new_first > 1) new_ranges.emplace_back(new_first, j); @@ -370,7 +370,7 @@ void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualR size_t new_first = first; for (size_t j = first + 1; j < limit; ++j) { - if (comparator(res[j], res[new_first]) != 0) + if (cmp(res[j], res[new_first]) != 0) { if (j - new_first > 1) new_ranges.emplace_back(new_first, j); @@ -380,7 +380,7 @@ void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualR size_t new_last = limit; for (size_t j = limit; j < last; ++j) { - if (comparator(res[j], res[new_first]) == 0) + if (cmp(res[j], res[new_first]) == 0) { std::swap(res[j], res[new_last]); ++new_last; @@ -393,21 +393,21 @@ void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualR void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res) const { - getPermutationImpl(limit, res, cmp(*this, reverse)); + getPermutationImpl(limit, res, Cmp(*this, reverse)); } void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res, EqualRanges & equal_ranges) const { - updatePermutationImpl(limit, res, equal_ranges, cmp(*this, reverse)); + updatePermutationImpl(limit, res, equal_ranges, Cmp(*this, reverse)); } -struct ColumnString::cmpWithCollation +struct ColumnString::CmpWithCollation { const ColumnString & parent; const Collator & collator; bool reverse; - cmpWithCollation(const ColumnString & parent_, const Collator & collator_, bool reverse_=false) : parent(parent_), collator(collator_), reverse(reverse_) {} + CmpWithCollation(const ColumnString & parent_, const Collator & collator_, bool reverse_=false) : parent(parent_), collator(collator_), reverse(reverse_) {} int operator()(size_t lhs, size_t rhs) const { @@ -419,17 +419,16 @@ struct ColumnString::cmpWithCollation } }; -void ColumnString::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, Permutation & res) const +void ColumnString::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res) const { - getPermutationImpl(limit, res, cmpWithCollation(*this, collator, reverse)); + getPermutationImpl(limit, res, CmpWithCollation(*this, collator, reverse)); } void ColumnString::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const { - updatePermutationImpl(limit, res, equal_ranges, cmpWithCollation(*this, collator, reverse)); + updatePermutationImpl(limit, res, equal_ranges, CmpWithCollation(*this, collator, reverse)); } - ColumnPtr ColumnString::replicate(const Offsets & replicate_offsets) const { size_t col_size = size(); @@ -498,7 +497,7 @@ void ColumnString::getExtremes(Field & min, Field & max) const size_t min_idx = 0; size_t max_idx = 0; - cmp cmp_op(*this); + Cmp cmp_op(*this); for (size_t i = 1; i < col_size; ++i) { @@ -513,7 +512,7 @@ void ColumnString::getExtremes(Field & min, Field & max) const } -int ColumnString::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, const Collator & collator) const +int ColumnString::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, int, const Collator & collator) const { const ColumnString & rhs = assert_cast(rhs_); diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index c91d982f126..1e6f60e63b3 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -42,18 +42,18 @@ private: /// Size of i-th element, including terminating zero. size_t ALWAYS_INLINE sizeAt(ssize_t i) const { return offsets[i] - offsets[i - 1]; } - struct cmp; + struct Cmp; - struct cmpWithCollation; + struct CmpWithCollation; ColumnString() = default; ColumnString(const ColumnString & src); - template - void getPermutationImpl(size_t limit, Permutation & res, Cmp comparator) const; + template + void getPermutationImpl(size_t limit, Permutation & res, Comparator cmp) const; - template - void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const; + template + void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Comparator cmp) const; public: const char * getFamilyName() const override { return "String"; } @@ -233,16 +233,16 @@ public: int direction, int nan_direction_hint) const override; /// Variant of compareAt for string comparison with respect of collation. - int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, const Collator & collator) const; + int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, int, const Collator & collator) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const override; /// Sorting with respect of collation. - void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, Permutation & res) const; + void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res) const override; - void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const; + void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const override; ColumnPtr replicate(const Offsets & replicate_offsets) const override; @@ -274,6 +274,8 @@ public: // Throws an exception if offsets/chars are messed up void validate() const; + + bool isCollationSupported() const override { return true; } }; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 98a6611edb7..f588762fb67 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -275,16 +275,27 @@ MutableColumns ColumnTuple::scatter(ColumnIndex num_columns, const Selector & se return res; } -int ColumnTuple::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +int ColumnTuple::compareAtImpl(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator * collator) const { const size_t tuple_size = columns.size(); for (size_t i = 0; i < tuple_size; ++i) - if (int res = columns[i]->compareAt(n, m, *assert_cast(rhs).columns[i], nan_direction_hint)) + { + int res; + if (collator && columns[i]->isCollationSupported()) + res = columns[i]->compareAtWithCollation(n, m, *assert_cast(rhs).columns[i], nan_direction_hint, *collator); + else + res = columns[i]->compareAt(n, m, *assert_cast(rhs).columns[i], nan_direction_hint); + if (res) return res; - + } return 0; } +int ColumnTuple::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +{ + return compareAtImpl(n, m, rhs, nan_direction_hint); +} + void ColumnTuple::compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const @@ -293,14 +304,20 @@ void ColumnTuple::compareColumn(const IColumn & rhs, size_t rhs_row_num, compare_results, direction, nan_direction_hint); } -template +int ColumnTuple::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator & collator) const +{ + return compareAtImpl(n, m, rhs, nan_direction_hint, &collator); +} + struct ColumnTuple::Less { TupleColumns columns; int nan_direction_hint; + bool reverse; + const Collator * collator; - Less(const TupleColumns & columns_, int nan_direction_hint_) - : columns(columns_), nan_direction_hint(nan_direction_hint_) + Less(const TupleColumns & columns_, int nan_direction_hint_, bool reverse_=false, const Collator * collator_=nullptr) + : columns(columns_), nan_direction_hint(nan_direction_hint_), reverse(reverse_), collator(collator_) { } @@ -308,17 +325,22 @@ struct ColumnTuple::Less { for (const auto & column : columns) { - int res = column->compareAt(a, b, *column, nan_direction_hint); + int res; + if (collator && column->isCollationSupported()) + res = column->compareAtWithCollation(a, b, *column, nan_direction_hint, *collator); + else + res = column->compareAt(a, b, *column, nan_direction_hint); if (res < 0) - return positive; + return !reverse; else if (res > 0) - return !positive; + return reverse; } return false; } }; -void ColumnTuple::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +template +void ColumnTuple::getPermutationImpl(size_t limit, Permutation & res, LessOperator less) const { size_t rows = size(); res.resize(rows); @@ -330,28 +352,25 @@ void ColumnTuple::getPermutation(bool reverse, size_t limit, int nan_direction_h if (limit) { - if (reverse) - std::partial_sort(res.begin(), res.begin() + limit, res.end(), Less(columns, nan_direction_hint)); - else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), Less(columns, nan_direction_hint)); + std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); } else { - if (reverse) - std::sort(res.begin(), res.end(), Less(columns, nan_direction_hint)); - else - std::sort(res.begin(), res.end(), Less(columns, nan_direction_hint)); + std::sort(res.begin(), res.end(), less); } } -void ColumnTuple::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const +void ColumnTuple::updatePermutationImpl(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges, const Collator * collator) const { if (equal_ranges.empty()) return; for (const auto & column : columns) { - column->updatePermutation(reverse, limit, nan_direction_hint, res, equal_ranges); + if (collator && column->isCollationSupported()) + column->updatePermutationWithCollation(*collator, reverse, limit, nan_direction_hint, res, equal_ranges); + else + column->updatePermutation(reverse, limit, nan_direction_hint, res, equal_ranges); while (limit && !equal_ranges.empty() && limit <= equal_ranges.back().first) equal_ranges.pop_back(); @@ -361,6 +380,26 @@ void ColumnTuple::updatePermutation(bool reverse, size_t limit, int nan_directio } } +void ColumnTuple::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +{ + getPermutationImpl(limit, res, Less(columns, nan_direction_hint, reverse)); +} + +void ColumnTuple::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const +{ + updatePermutationImpl(reverse, limit, nan_direction_hint, res, equal_ranges); +} + +void ColumnTuple::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +{ + getPermutationImpl(limit, res, Less(columns, nan_direction_hint, reverse, &collator)); +} + +void ColumnTuple::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_ranges) const +{ + updatePermutationImpl(reverse, limit, nan_direction_hint, res, equal_ranges, &collator); +} + void ColumnTuple::gather(ColumnGathererStream & gatherer) { gatherer.gather(*this); @@ -433,5 +472,15 @@ bool ColumnTuple::structureEquals(const IColumn & rhs) const return false; } +bool ColumnTuple::isCollationSupported() const +{ + for (const auto& column : columns) + { + if (column->isCollationSupported()) + return true; + } + return false; +} + } diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index e8dfd4c8e44..c34768d85a4 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -20,7 +20,6 @@ private: using TupleColumns = std::vector; TupleColumns columns; - template struct Less; explicit ColumnTuple(MutableColumns && columns); @@ -75,15 +74,19 @@ public: void compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; + int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator & collator) const override; void getExtremes(Field & min, Field & max) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; - void updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_range) const override; + void updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override; + void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; + void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override; void reserve(size_t n) override; size_t byteSize() const override; size_t allocatedBytes() const override; void protect() override; void forEachSubcolumn(ColumnCallback callback) override; bool structureEquals(const IColumn & rhs) const override; + bool isCollationSupported() const override; size_t tupleSize() const { return columns.size(); } @@ -94,6 +97,15 @@ public: Columns getColumnsCopy() const { return {columns.begin(), columns.end()}; } const ColumnPtr & getColumnPtr(size_t idx) const { return columns[idx]; } + +private: + int compareAtImpl(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator * collator=nullptr) const; + + template + void getPermutationImpl(size_t limit, Permutation & res, LessOperator less) const; + + void updatePermutationImpl( + bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges, const Collator * collator=nullptr) const; }; diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 14e6a9d7eed..6dbcfacefe9 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -9,7 +9,7 @@ class SipHash; - +class Collator; namespace DB { @@ -18,6 +18,7 @@ namespace ErrorCodes { extern const int CANNOT_GET_SIZE_OF_FIELD; extern const int NOT_IMPLEMENTED; + extern const int BAD_COLLATION; } class Arena; @@ -250,6 +251,12 @@ public: */ virtual int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const = 0; + /// Equivalent to compareAt, but collator is used to compare values. + virtual int compareAtWithCollation(size_t, size_t, const IColumn &, int, const Collator &) const + { + throw Exception("Collations could be specified only for String, LowCardinality(String), Nullable(String) or for Array or Tuple, containing it.", ErrorCodes::BAD_COLLATION); + } + /// Compare the whole column with single value from rhs column. /// If row_indexes is nullptr, it's ignored. Otherwise, it is a set of rows to compare. /// compare_results[i] will be equal to compareAt(row_indexes[i], rhs_row_num, rhs, nan_direction_hint) * direction @@ -277,6 +284,18 @@ public: */ virtual void updatePermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_ranges) const = 0; + /** Equivalent to getPermutation and updatePermutation but collator is used to compare values. + * Supported for String, LowCardinality(String), Nullable(String) and for Array and Tuple, containing them. + */ + virtual void getPermutationWithCollation(const Collator &, bool, size_t, int, Permutation &) const + { + throw Exception("Collations could be specified only for String, LowCardinality(String), Nullable(String) or for Array or Tuple, containing them.", ErrorCodes::BAD_COLLATION); + } + virtual void updatePermutationWithCollation(const Collator &, bool, size_t, int, Permutation &, EqualRanges&) const + { + throw Exception("Collations could be specified only for String, LowCardinality(String), Nullable(String) or for Array or Tuple, containing them.", ErrorCodes::BAD_COLLATION); + } + /** Copies each element according offsets parameter. * (i-th element should be copied offsets[i] - offsets[i - 1] times.) * It is necessary in ARRAY JOIN operation. @@ -402,6 +421,8 @@ public: virtual bool lowCardinality() const { return false; } + virtual bool isCollationSupported() const { return false; } + virtual ~IColumn() = default; IColumn() = default; IColumn(const IColumn &) = default; diff --git a/src/Core/SortCursor.h b/src/Core/SortCursor.h index 4c90cc723bf..7a222f70199 100644 --- a/src/Core/SortCursor.h +++ b/src/Core/SortCursor.h @@ -96,7 +96,7 @@ struct SortCursorImpl : column_desc.column_number; sort_columns.push_back(columns[column_number].get()); - need_collation[j] = desc[j].collator != nullptr && typeid_cast(sort_columns.back()); /// TODO Nullable(String) + need_collation[j] = desc[j].collator != nullptr && sort_columns.back()->isCollationSupported(); /// TODO Nullable(String) has_collation |= need_collation[j]; } @@ -201,10 +201,7 @@ struct SortCursorWithCollation : SortCursorHelper int nulls_direction = desc.nulls_direction; int res; if (impl->need_collation[i]) - { - const ColumnString & column_string = assert_cast(*impl->sort_columns[i]); - res = column_string.compareAtWithCollation(lhs_pos, rhs_pos, *(rhs.impl->sort_columns[i]), *impl->desc[i].collator); - } + res = impl->sort_columns[i]->compareAtWithCollation(lhs_pos, rhs_pos, *(rhs.impl->sort_columns[i]), nulls_direction, *impl->desc[i].collator); else res = impl->sort_columns[i]->compareAt(lhs_pos, rhs_pos, *(rhs.impl->sort_columns[i]), nulls_direction); diff --git a/src/Interpreters/sortBlock.cpp b/src/Interpreters/sortBlock.cpp index 5d114c746e5..edf911fa61c 100644 --- a/src/Interpreters/sortBlock.cpp +++ b/src/Interpreters/sortBlock.cpp @@ -22,24 +22,6 @@ static bool isCollationRequired(const SortColumnDescription & description) return description.collator != nullptr; } -static bool isCollationSupported(const IColumn * column) -{ - if (column->getDataType() == TypeIndex::String) - return true; - - if (column->getDataType() == TypeIndex::Nullable) - { - const ColumnNullable * column_nullable = assert_cast(column); - return isCollationSupported(&column_nullable->getNestedColumn()); - } - - if (column->getDataType() == TypeIndex::LowCardinality) - { - const ColumnLowCardinality * column_low_cardinality = assert_cast(column); - return isCollationSupported(column_low_cardinality->getDictionary().getNestedColumn().get()); - } - return false; -} ColumnsWithSortDescriptions getColumnsWithSortDescription(const Block & block, const SortDescription & description) { @@ -106,8 +88,7 @@ struct PartialSortingLessWithCollation } else if (isCollationRequired(elem.description)) { - const ColumnString & column_string = assert_cast(*elem.column); - res = column_string.compareAtWithCollation(a, b, *elem.column, *elem.description.collator); + res = elem.column->compareAtWithCollation(a, b, *elem.column, elem.description.nulls_direction, *elem.description.collator); } else res = elem.column->compareAt(a, b, *elem.column, elem.description.nulls_direction); @@ -139,18 +120,13 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) bool is_column_const = false; if (isCollationRequired(description[0])) { - /// Check if column supports collations - if (!isCollationSupported(column)) - throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); + if (!column->isCollationSupported()) + throw Exception("Collations could be specified only for String, LowCardinality(String), Nullable(String) or for Array or Tuple, containing them.", ErrorCodes::BAD_COLLATION); - if (const ColumnString * column_string = checkAndGetColumn(column)) - column_string->getPermutationWithCollation(*description[0].collator, reverse, limit, perm); - else if (const ColumnNullable * column_nullable = checkAndGetColumn(column)) - column_nullable->getPermutationWithCollation(*description[0].collator, reverse, limit, description[0].nulls_direction, perm); - else if (const ColumnLowCardinality * column_low_cardinality = checkAndGetColumn(column)) - column_low_cardinality->getPermutationWithCollation(*description[0].collator, reverse, limit, description[0].nulls_direction, perm); - else if (isColumnConst(*column)) + if (isColumnConst(*column)) is_column_const = true; + else + column->getPermutationWithCollation(*description[0].collator, reverse, limit, description[0].nulls_direction, perm); } else if (!isColumnConst(*column)) { @@ -186,8 +162,8 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) const IColumn * column = columns_with_sort_desc[i].column; if (isCollationRequired(description[i])) { - if (!isCollationSupported(column)) - throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); + if (!column->isCollationSupported()) + throw Exception("Collations could be specified only for String, LowCardinality(String), Nullable(String) or for Array or Tuple, containing them.", ErrorCodes::BAD_COLLATION); need_collation = true; } @@ -210,20 +186,8 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) if (isCollationRequired(column.description)) { - if (const ColumnString * column_string = checkAndGetColumn(column.column)) - column_string->updatePermutationWithCollation( - *column.description.collator, - column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); - - else if (const ColumnNullable * column_nullable = checkAndGetColumn(column.column)) - column_nullable->updatePermutationWithCollation( - *column.description.collator, - column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); - - else if (const ColumnLowCardinality * column_low_cardinality = checkAndGetColumn(column.column)) - column_low_cardinality->updatePermutationWithCollation( - *column.description.collator, - column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); + column.column->updatePermutationWithCollation( + *column.description.collator, column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); } else { diff --git a/tests/queries/0_stateless/01532_collate_in_low_cardinality.reference b/tests/queries/0_stateless/01532_collate_in_low_cardinality.reference index b7a4830f9cf..fbffea8df5a 100644 --- a/tests/queries/0_stateless/01532_collate_in_low_cardinality.reference +++ b/tests/queries/0_stateless/01532_collate_in_low_cardinality.reference @@ -26,3 +26,39 @@ Order by tuple with collate 2 А 2 я 2 Я +Order by without collate +1 Ё +2 А +2 Я +1 а +2 я +1 ё +1 \N +2 \N +Order by with collate +1 а +2 А +1 ё +1 Ё +2 я +2 Я +1 \N +2 \N +Order by tuple without collate +1 Ё +1 а +1 ё +1 \N +2 А +2 Я +2 я +2 \N +Order by tuple with collate +1 а +1 ё +1 Ё +1 \N +2 А +2 я +2 Я +2 \N diff --git a/tests/queries/0_stateless/01532_collate_in_low_cardinality.sql b/tests/queries/0_stateless/01532_collate_in_low_cardinality.sql index 0f4194ee671..b6fba26eb2d 100644 --- a/tests/queries/0_stateless/01532_collate_in_low_cardinality.sql +++ b/tests/queries/0_stateless/01532_collate_in_low_cardinality.sql @@ -1,8 +1,12 @@ DROP TABLE IF EXISTS test_collate; +DROP TABLE IF EXISTS test_collate_null; CREATE TABLE test_collate (x UInt32, s LowCardinality(String)) ENGINE=Memory(); +CREATE TABLE test_collate_null (x UInt32, s LowCardinality(Nullable(String))) ENGINE=Memory(); INSERT INTO test_collate VALUES (1, 'Ё'), (1, 'ё'), (1, 'а'), (2, 'А'), (2, 'я'), (2, 'Я'); +INSERT INTO test_collate_null VALUES (1, 'Ё'), (1, 'ё'), (1, 'а'), (2, 'А'), (2, 'я'), (2, 'Я'), (1, null), (2, null); + SELECT 'Order by without collate'; SELECT * FROM test_collate ORDER BY s; @@ -14,5 +18,16 @@ SELECT * FROM test_collate ORDER BY x, s; SELECT 'Order by tuple with collate'; SELECT * FROM test_collate ORDER BY x, s COLLATE 'ru'; -DROP TABLE test_collate; +SELECT 'Order by without collate'; +SELECT * FROM test_collate_null ORDER BY s; +SELECT 'Order by with collate'; +SELECT * FROM test_collate_null ORDER BY s COLLATE 'ru'; +SELECT 'Order by tuple without collate'; +SELECT * FROM test_collate_null ORDER BY x, s; +SELECT 'Order by tuple with collate'; +SELECT * FROM test_collate_null ORDER BY x, s COLLATE 'ru'; + + +DROP TABLE test_collate; +DROP TABLE test_collate_null; diff --git a/tests/queries/0_stateless/01542_collate_in_array.reference b/tests/queries/0_stateless/01542_collate_in_array.reference new file mode 100644 index 00000000000..2c5a23066f3 --- /dev/null +++ b/tests/queries/0_stateless/01542_collate_in_array.reference @@ -0,0 +1,50 @@ +1 ['а'] +2 ['А'] +1 ['ё'] +1 ['ё','а'] +2 ['ё','а','а'] +1 ['ё','я'] +1 ['Ё'] +2 ['я','а'] +2 ['Я'] + +1 ['а'] +1 ['ё'] +1 ['ё','а'] +1 ['ё','я'] +1 ['Ё'] +2 ['А'] +2 ['ё','а','а'] +2 ['я','а'] +2 ['Я'] + +1 ['а'] +2 ['А'] +1 ['ё'] +1 ['ё','а'] +2 ['ё','а','а',NULL] +1 ['ё',NULL,'я'] +1 ['Ё'] +2 ['я'] +2 [NULL,'Я'] + +1 ['а'] +1 ['ё'] +1 ['ё','а'] +1 ['ё',NULL,'я'] +1 ['Ё'] +2 ['А'] +2 ['ё','а','а',NULL] +2 ['я'] +2 [NULL,'Я'] + +2 [['а','а'],['я','ё']] +1 [['а','Ё'],['ё','я']] +1 [['а','я'],['а','ё']] +2 [['ё']] + +1 [['а','Ё'],['ё','я']] +1 [['а','я'],['а','ё']] +2 [['а','а'],['я','ё']] +2 [['ё']] + diff --git a/tests/queries/0_stateless/01542_collate_in_array.sql b/tests/queries/0_stateless/01542_collate_in_array.sql new file mode 100644 index 00000000000..dd0ec769e7d --- /dev/null +++ b/tests/queries/0_stateless/01542_collate_in_array.sql @@ -0,0 +1,34 @@ +DROP TABLE IF EXISTS collate_test1; +DROP TABLE IF EXISTS collate_test2; +DROP TABLE IF EXISTS collate_test3; + +CREATE TABLE collate_test1 (x UInt32, s Array(String)) ENGINE=Memory(); +CREATE TABLE collate_test2 (x UInt32, s Array(LowCardinality(Nullable(String)))) ENGINE=Memory(); +CREATE TABLE collate_test3 (x UInt32, s Array(Array(String))) ENGINE=Memory(); + +INSERT INTO collate_test1 VALUES (1, ['Ё']), (1, ['ё']), (1, ['а']), (2, ['А']), (2, ['я', 'а']), (2, ['Я']), (1, ['ё','а']), (1, ['ё', 'я']), (2, ['ё', 'а', 'а']); +INSERT INTO collate_test2 VALUES (1, ['Ё']), (1, ['ё']), (1, ['а']), (2, ['А']), (2, ['я']), (2, [null, 'Я']), (1, ['ё','а']), (1, ['ё', null, 'я']), (2, ['ё', 'а', 'а', null]); +INSERT INTO collate_test3 VALUES (1, [['а', 'я'], ['а', 'ё']]), (1, [['а', 'Ё'], ['ё', 'я']]), (2, [['ё']]), (2, [['а', 'а'], ['я', 'ё']]); + +SELECT * FROM collate_test1 ORDER BY s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test1 ORDER BY x, s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test2 ORDER BY s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test2 ORDER BY x, s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test3 ORDER BY s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test3 ORDER BY x, s COLLATE 'ru'; +SELECT ''; + +DROP TABLE collate_test1; +DROP TABLE collate_test2; +DROP TABLE collate_test3; + diff --git a/tests/queries/0_stateless/01543_collate_in_tuple.reference b/tests/queries/0_stateless/01543_collate_in_tuple.reference new file mode 100644 index 00000000000..fe8f935f0a6 --- /dev/null +++ b/tests/queries/0_stateless/01543_collate_in_tuple.reference @@ -0,0 +1,60 @@ +1 (1,'а') +1 (1,'ё') +1 (1,'Ё') +2 (1,'я') +1 (2,'а') +2 (2,'А') +2 (2,'Я') +1 (3,'я') + +1 (1,'а') +1 (1,'ё') +1 (1,'Ё') +1 (2,'а') +1 (3,'я') +2 (1,'я') +2 (2,'А') +2 (2,'Я') + +1 (1,'а') +1 (1,'ё') +1 (1,'Ё') +2 (1,'я') +1 (1,NULL) +2 (2,'А') +2 (2,'Я') +1 (2,NULL) +2 (2,NULL) +1 (3,'я') + +1 (1,'а') +1 (1,'ё') +1 (1,'Ё') +1 (1,NULL) +1 (2,NULL) +1 (3,'я') +2 (1,'я') +2 (2,'А') +2 (2,'Я') +2 (2,NULL) + +2 (1,(1,['А'])) +2 (1,(1,['ё','а','а'])) +1 (1,(1,['Ё'])) +2 (1,(1,['Я'])) +1 (1,(2,['а'])) +1 (1,(2,['ё','я'])) +1 (2,(1,['ё'])) +1 (2,(1,['ё','а'])) +2 (2,(1,['я'])) + +1 (1,(1,['Ё'])) +1 (1,(2,['а'])) +1 (1,(2,['ё','я'])) +1 (2,(1,['ё'])) +1 (2,(1,['ё','а'])) +2 (1,(1,['А'])) +2 (1,(1,['ё','а','а'])) +2 (1,(1,['Я'])) +2 (2,(1,['я'])) + diff --git a/tests/queries/0_stateless/01543_collate_in_tuple.sql b/tests/queries/0_stateless/01543_collate_in_tuple.sql new file mode 100644 index 00000000000..17d9426cf45 --- /dev/null +++ b/tests/queries/0_stateless/01543_collate_in_tuple.sql @@ -0,0 +1,34 @@ +DROP TABLE IF EXISTS collate_test1; +DROP TABLE IF EXISTS collate_test2; +DROP TABLE IF EXISTS collate_test3; + +CREATE TABLE collate_test1 (x UInt32, s Tuple(UInt32, String)) ENGINE=Memory(); +CREATE TABLE collate_test2 (x UInt32, s Tuple(UInt32, LowCardinality(Nullable(String)))) ENGINE=Memory(); +CREATE TABLE collate_test3 (x UInt32, s Tuple(UInt32, Tuple(UInt32, Array(String)))) ENGINE=Memory(); + +INSERT INTO collate_test1 VALUES (1, (1, 'Ё')), (1, (1, 'ё')), (1, (1, 'а')), (2, (2, 'А')), (2, (1, 'я')), (2, (2, 'Я')), (1, (2,'а')), (1, (3, 'я')); +INSERT INTO collate_test2 VALUES (1, (1, 'Ё')), (1, (1, 'ё')), (1, (1, 'а')), (2, (2, 'А')), (2, (1, 'я')), (2, (2, 'Я')), (1, (2, null)), (1, (3, 'я')), (1, (1, null)), (2, (2, null)); +INSERT INTO collate_test3 VALUES (1, (1, (1, ['Ё']))), (1, (2, (1, ['ё']))), (1, (1, (2, ['а']))), (2, (1, (1, ['А']))), (2, (2, (1, ['я']))), (2, (1, (1, ['Я']))), (1, (2, (1, ['ё','а']))), (1, (1, (2, ['ё', 'я']))), (2, (1, (1, ['ё', 'а', 'а']))); + +SELECT * FROM collate_test1 ORDER BY s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test1 ORDER BY x, s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test2 ORDER BY s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test2 ORDER BY x, s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test3 ORDER BY s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test3 ORDER BY x, s COLLATE 'ru'; +SELECT ''; + +DROP TABLE collate_test1; +DROP TABLE collate_test2; +DROP TABLE collate_test3; + From cf3f39ed29bf5a127c1dcc5c97ded68fb75befc8 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 29 Oct 2020 14:37:00 +0300 Subject: [PATCH 139/314] Small changes --- src/Columns/ColumnLowCardinality.cpp | 2 -- src/Columns/ColumnNullable.cpp | 4 ---- 2 files changed, 6 deletions(-) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 37e97da88b9..3f03734b738 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -316,9 +316,7 @@ void ColumnLowCardinality::getPermutationImpl(bool reverse, size_t limit, int na size_t unique_limit = getDictionary().size(); Permutation unique_perm; if (collator) - { getDictionary().getNestedColumn()->getPermutationWithCollation(*collator, reverse, unique_limit, nan_direction_hint, unique_perm); - } else getDictionary().getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm); diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index cbb82264694..4f2117b1405 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -275,9 +275,7 @@ void ColumnNullable::getPermutationImpl(bool reverse, size_t limit, int null_dir /// Cannot pass limit because of unknown amount of NULLs. if (collator) - { getNestedColumn().getPermutationWithCollation(*collator, reverse, 0, null_direction_hint, res); - } else getNestedColumn().getPermutation(reverse, 0, null_direction_hint, res); @@ -453,9 +451,7 @@ void ColumnNullable::updatePermutationImpl(bool reverse, size_t limit, int null_ } if (collator) - { getNestedColumn().updatePermutationWithCollation(*collator, reverse, limit, null_direction_hint, res, new_ranges); - } else getNestedColumn().updatePermutation(reverse, limit, null_direction_hint, res, new_ranges); From bcd660bb57862b2aae0572518c1ecde2be59c21b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 30 Oct 2020 08:35:18 +0300 Subject: [PATCH 140/314] Minor fixes --- src/Columns/ColumnArray.cpp | 5 +++-- src/Columns/ColumnConst.h | 2 ++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index c061dd50642..b420d337701 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -368,13 +368,14 @@ void ColumnArray::compareColumn(const IColumn & rhs, size_t rhs_row_num, compare_results, direction, nan_direction_hint); } -struct ColumnArray::Cmp { +struct ColumnArray::Cmp +{ const ColumnArray & parent; int nan_direction_hint; bool reverse; const Collator * collator; - Cmp(const ColumnArray & parent_, int nan_direction_hint_, bool reverse_=false, const Collator * collator_=nullptr) + Cmp(const ColumnArray & parent_, int nan_direction_hint_, bool reverse_ = false, const Collator * collator_ = nullptr) : parent(parent_), nan_direction_hint(nan_direction_hint_), reverse(reverse_), collator(collator_) {} int operator()(size_t lhs, size_t rhs) const diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index 4942d27b6c9..d7a8842bf01 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -248,6 +248,8 @@ public: /// The constant value. It is valid even if the size of the column is 0. template T getValue() const { return getField().safeGet>(); } + + bool isCollationSupported() const override { return true; } }; } From cd86f98aec6ff47185d6972d3509e508db4fdd3c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 30 Oct 2020 08:36:27 +0300 Subject: [PATCH 141/314] Minor fixes --- src/Columns/ColumnConst.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index d7a8842bf01..3680926cd9b 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -249,7 +249,7 @@ public: template T getValue() const { return getField().safeGet>(); } - bool isCollationSupported() const override { return true; } + bool isCollationSupported() const override { return data->isCollationSupported(); } }; } From 9868b58531b7ce726e11e71a0b0b068cbb73cd06 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 31 Oct 2020 03:59:58 +0300 Subject: [PATCH 142/314] Minor change --- src/Columns/ColumnString.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 9ea12041d85..477c098f067 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -407,7 +407,8 @@ struct ColumnString::CmpWithCollation const Collator & collator; bool reverse; - CmpWithCollation(const ColumnString & parent_, const Collator & collator_, bool reverse_=false) : parent(parent_), collator(collator_), reverse(reverse_) {} + CmpWithCollation(const ColumnString & parent_, const Collator & collator_, bool reverse_ = false) + : parent(parent_), collator(collator_), reverse(reverse_) {} int operator()(size_t lhs, size_t rhs) const { From 726be794a4546101c078976427e77c7ce657876e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Nov 2020 17:14:34 +0300 Subject: [PATCH 143/314] Update ExpressionAnalyzer.cpp --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 516c12b2be7..45230c53e81 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -835,8 +835,8 @@ bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); - step.required_output.push_back(select_query->having()->getColumnName()); getRootActionsForHaving(select_query->having(), only_types, step.actions()); + step.required_output.push_back(select_query->having()->getColumnName()); return true; } From 822bbcfdba6544dbcddd2f6985b92a3827504862 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 3 Nov 2020 17:25:52 +0300 Subject: [PATCH 144/314] Make Cmp with template --- src/Columns/ColumnArray.cpp | 32 ++++++++++++++++++++++---------- src/Columns/ColumnArray.h | 1 + src/Columns/ColumnString.cpp | 35 +++++++++++++++++++++++------------ src/Columns/ColumnString.h | 2 ++ src/Columns/ColumnTuple.cpp | 20 +++++++++++++------- src/Columns/ColumnTuple.h | 1 + 6 files changed, 62 insertions(+), 29 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index b420d337701..f03a51e0681 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -368,15 +368,14 @@ void ColumnArray::compareColumn(const IColumn & rhs, size_t rhs_row_num, compare_results, direction, nan_direction_hint); } -struct ColumnArray::Cmp -{ +template +struct ColumnArray::Cmp { const ColumnArray & parent; int nan_direction_hint; - bool reverse; const Collator * collator; - Cmp(const ColumnArray & parent_, int nan_direction_hint_, bool reverse_ = false, const Collator * collator_ = nullptr) - : parent(parent_), nan_direction_hint(nan_direction_hint_), reverse(reverse_), collator(collator_) {} + Cmp(const ColumnArray & parent_, int nan_direction_hint_, const Collator * collator_=nullptr) + : parent(parent_), nan_direction_hint(nan_direction_hint_), collator(collator_) {} int operator()(size_t lhs, size_t rhs) const { @@ -385,7 +384,7 @@ struct ColumnArray::Cmp res = parent.compareAtWithCollation(lhs, rhs, parent, nan_direction_hint, *collator); else res = parent.compareAt(lhs, rhs, parent, nan_direction_hint); - return reverse ? -res : res; + return positive ? res : -res; } }; @@ -866,22 +865,35 @@ void ColumnArray::updatePermutationImpl(size_t limit, Permutation & res, EqualRa void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const { - getPermutationImpl(limit, res, Cmp(*this, nan_direction_hint, reverse)); + if (reverse) + getPermutationImpl(limit, res, Cmp(*this, nan_direction_hint)); + else + getPermutationImpl(limit, res, Cmp(*this, nan_direction_hint)); + } void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_range) const { - updatePermutationImpl(limit, res, equal_range, Cmp(*this, nan_direction_hint, reverse)); + if (reverse) + updatePermutationImpl(limit, res, equal_range, Cmp(*this, nan_direction_hint)); + else + updatePermutationImpl(limit, res, equal_range, Cmp(*this, nan_direction_hint)); } void ColumnArray::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const { - getPermutationImpl(limit, res, Cmp(*this, nan_direction_hint, reverse, &collator)); + if (reverse) + getPermutationImpl(limit, res, Cmp(*this, nan_direction_hint, &collator)); + else + getPermutationImpl(limit, res, Cmp(*this, nan_direction_hint, &collator)); } void ColumnArray::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_range) const { - updatePermutationImpl(limit, res, equal_range, Cmp(*this, nan_direction_hint, reverse, &collator)); + if (reverse) + updatePermutationImpl(limit, res, equal_range, Cmp(*this, nan_direction_hint, &collator)); + else + updatePermutationImpl(limit, res, equal_range, Cmp(*this, nan_direction_hint, &collator)); } ColumnPtr ColumnArray::replicate(const Offsets & replicate_offsets) const diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 028eaba73c5..8a02af92dce 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -183,6 +183,7 @@ private: template void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_range, Comparator cmp) const; + template struct Cmp; }; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 477c098f067..23798f64a9c 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -284,18 +284,18 @@ void ColumnString::compareColumn( compare_results, direction, nan_direction_hint); } +template struct ColumnString::Cmp { const ColumnString & parent; - bool reverse; - explicit Cmp(const ColumnString & parent_, bool reverse_=false) : parent(parent_), reverse(reverse_) {} + explicit Cmp(const ColumnString & parent_) : parent(parent_) {} int operator()(size_t lhs, size_t rhs) const { int res = memcmpSmallAllowOverflow15( parent.chars.data() + parent.offsetAt(lhs), parent.sizeAt(lhs) - 1, parent.chars.data() + parent.offsetAt(rhs), parent.sizeAt(rhs) - 1); - return reverse ? -res : res; + return positive ? res : -res; } }; @@ -393,22 +393,27 @@ void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualR void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res) const { - getPermutationImpl(limit, res, Cmp(*this, reverse)); + if (reverse) + getPermutationImpl(limit, res, Cmp(*this)); + else + getPermutationImpl(limit, res, Cmp(*this)); } void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res, EqualRanges & equal_ranges) const { - updatePermutationImpl(limit, res, equal_ranges, Cmp(*this, reverse)); + if (reverse) + updatePermutationImpl(limit, res, equal_ranges, Cmp(*this)); + else + updatePermutationImpl(limit, res, equal_ranges, Cmp(*this)); } +template struct ColumnString::CmpWithCollation { const ColumnString & parent; const Collator & collator; - bool reverse; - CmpWithCollation(const ColumnString & parent_, const Collator & collator_, bool reverse_ = false) - : parent(parent_), collator(collator_), reverse(reverse_) {} + CmpWithCollation(const ColumnString & parent_, const Collator & collator_) : parent(parent_), collator(collator_) {} int operator()(size_t lhs, size_t rhs) const { @@ -416,18 +421,24 @@ struct ColumnString::CmpWithCollation reinterpret_cast(&parent.chars[parent.offsetAt(lhs)]), parent.sizeAt(lhs), reinterpret_cast(&parent.chars[parent.offsetAt(rhs)]), parent.sizeAt(rhs)); - return reverse ? -res : res; + return positive ? res : -res; } }; void ColumnString::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res) const { - getPermutationImpl(limit, res, CmpWithCollation(*this, collator, reverse)); + if (reverse) + getPermutationImpl(limit, res, CmpWithCollation(*this, collator)); + else + getPermutationImpl(limit, res, CmpWithCollation(*this, collator)); } void ColumnString::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const { - updatePermutationImpl(limit, res, equal_ranges, CmpWithCollation(*this, collator, reverse)); + if (reverse) + updatePermutationImpl(limit, res, equal_ranges, CmpWithCollation(*this, collator)); + else + updatePermutationImpl(limit, res, equal_ranges, CmpWithCollation(*this, collator)); } ColumnPtr ColumnString::replicate(const Offsets & replicate_offsets) const @@ -498,7 +509,7 @@ void ColumnString::getExtremes(Field & min, Field & max) const size_t min_idx = 0; size_t max_idx = 0; - Cmp cmp_op(*this); + Cmp cmp_op(*this); for (size_t i = 1; i < col_size; ++i) { diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 1e6f60e63b3..b71751dbc4e 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -42,8 +42,10 @@ private: /// Size of i-th element, including terminating zero. size_t ALWAYS_INLINE sizeAt(ssize_t i) const { return offsets[i] - offsets[i - 1]; } + template struct Cmp; + template struct CmpWithCollation; ColumnString() = default; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index f588762fb67..d6e1ca982d6 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -309,15 +309,15 @@ int ColumnTuple::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, return compareAtImpl(n, m, rhs, nan_direction_hint, &collator); } +template struct ColumnTuple::Less { TupleColumns columns; int nan_direction_hint; - bool reverse; const Collator * collator; - Less(const TupleColumns & columns_, int nan_direction_hint_, bool reverse_=false, const Collator * collator_=nullptr) - : columns(columns_), nan_direction_hint(nan_direction_hint_), reverse(reverse_), collator(collator_) + Less(const TupleColumns & columns_, int nan_direction_hint_, const Collator * collator_=nullptr) + : columns(columns_), nan_direction_hint(nan_direction_hint_), collator(collator_) { } @@ -331,9 +331,9 @@ struct ColumnTuple::Less else res = column->compareAt(a, b, *column, nan_direction_hint); if (res < 0) - return !reverse; + return positive; else if (res > 0) - return reverse; + return !positive; } return false; } @@ -382,7 +382,10 @@ void ColumnTuple::updatePermutationImpl(bool reverse, size_t limit, int nan_dire void ColumnTuple::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const { - getPermutationImpl(limit, res, Less(columns, nan_direction_hint, reverse)); + if (reverse) + getPermutationImpl(limit, res, Less(columns, nan_direction_hint)); + else + getPermutationImpl(limit, res, Less(columns, nan_direction_hint)); } void ColumnTuple::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const @@ -392,7 +395,10 @@ void ColumnTuple::updatePermutation(bool reverse, size_t limit, int nan_directio void ColumnTuple::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const { - getPermutationImpl(limit, res, Less(columns, nan_direction_hint, reverse, &collator)); + if (reverse) + getPermutationImpl(limit, res, Less(columns, nan_direction_hint, &collator)); + else + getPermutationImpl(limit, res, Less(columns, nan_direction_hint, &collator)); } void ColumnTuple::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_ranges) const diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index c34768d85a4..0bee3463f2f 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -20,6 +20,7 @@ private: using TupleColumns = std::vector; TupleColumns columns; + template struct Less; explicit ColumnTuple(MutableColumns && columns); From 4245c470ac579d1d5f7b13083b32fd4a78c7a4fd Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 3 Nov 2020 17:37:54 +0300 Subject: [PATCH 145/314] fixes --- docker/test/performance-comparison/compare.sh | 11 ++++++----- docker/test/performance-comparison/entrypoint.sh | 3 +++ 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 20720e0b976..ca52fbd5891 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1085,7 +1085,7 @@ function upload_results # Surprisingly, clickhouse-client doesn't understand --host 127.0.0.1:9000 # so I have to do this instead. I tried to use Poco URI parser for this, # but it's also broken and can't parse host:port. - IFS=':' read host port <<<"${CHPC_DATABASE_URL}" + IFS=':' read -r host port <<<"${CHPC_DATABASE_URL}" upload_client=(clickhouse-client --host "${host}" @@ -1096,14 +1096,14 @@ function upload_results --config "ch/tests/config/client_config.xml" --database perftest -m - --date_time_input_format=best_effort) + "--date_time_input_format=best_effort") set +x # Don't show password in the log - cat "report/all-query-metrics.tsv" | "${upload_client[@]}" --query " + "${upload_client[@]}" --query " insert into query_metrics_tmp select toDate(event_time) event_date, - toDateTime('$(cd ch && git show -s --format=%ci "$SHA_TO_TEST" | cut -d' ' -f-2)') event_time, + toDateTime('$(cd right/ch && git show -s --format=%ci "$SHA_TO_TEST" | cut -d' ' -f-2)') event_time, $PR_TO_TEST pr_number, '$REF_SHA' old_sha, '$SHA_TO_TEST' new_sha, @@ -1120,7 +1120,8 @@ function upload_results test text, query_index int, query_display_name text') settings date_time_input_format='best_effort' format TSV - settings date_time_input_format='best_effort'" + settings date_time_input_format='best_effort' + " < "report/all-query-metrics.tsv" set -x } diff --git a/docker/test/performance-comparison/entrypoint.sh b/docker/test/performance-comparison/entrypoint.sh index ed2e542eadd..570a1c21514 100755 --- a/docker/test/performance-comparison/entrypoint.sh +++ b/docker/test/performance-comparison/entrypoint.sh @@ -121,6 +121,9 @@ set +e PATH="$(readlink -f right/)":"$PATH" export PATH +export REF_PR +export REF_SHA + # Start the main comparison script. { \ time ../download.sh "$REF_PR" "$REF_SHA" "$PR_TO_TEST" "$SHA_TO_TEST" && \ From b67465b010048d2c46cb0cf19e16aae9b13035fc Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 3 Nov 2020 18:00:51 +0300 Subject: [PATCH 146/314] Fix style --- src/Columns/ColumnArray.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index f03a51e0681..9b948236943 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -369,7 +369,8 @@ void ColumnArray::compareColumn(const IColumn & rhs, size_t rhs_row_num, } template -struct ColumnArray::Cmp { +struct ColumnArray::Cmp +{ const ColumnArray & parent; int nan_direction_hint; const Collator * collator; From 7c2361af456630e0dbe6dda9c99278e80ab1ddb5 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Tue, 3 Nov 2020 18:19:24 +0300 Subject: [PATCH 147/314] yet another logs --- src/Interpreters/AsynchronousMetricLog.h | 2 +- src/Interpreters/TraceLog.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/AsynchronousMetricLog.h b/src/Interpreters/AsynchronousMetricLog.h index 0c02244246e..30bac3f5a99 100644 --- a/src/Interpreters/AsynchronousMetricLog.h +++ b/src/Interpreters/AsynchronousMetricLog.h @@ -22,7 +22,7 @@ struct AsynchronousMetricLogElement { UInt16 event_date; time_t event_time; - UInt64 event_time_microseconds; + Decimal64 event_time_microseconds; std::string metric_name; double value; diff --git a/src/Interpreters/TraceLog.h b/src/Interpreters/TraceLog.h index d694a6201f7..9ee43bf32cc 100644 --- a/src/Interpreters/TraceLog.h +++ b/src/Interpreters/TraceLog.h @@ -18,7 +18,7 @@ struct TraceLogElement static const TraceDataType::Values trace_values; time_t event_time{}; - UInt64 event_time_microseconds{}; + Decimal64 event_time_microseconds{}; UInt64 timestamp_ns{}; TraceType trace_type{}; UInt64 thread_id{}; From faca9d9b91982eaecb1f46ee31d89bfc75d8ae39 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 3 Nov 2020 18:41:46 +0300 Subject: [PATCH 148/314] Check max bytes on disk --- src/Storages/MergeTree/MergeTreeData.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index fb7a7c02078..a06fbc338a4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2648,6 +2648,9 @@ void MergeTreeData::checkPartCanBeDropped(const ASTPtr & part_ast) auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Committed}); if (!part) throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in commited state", part_name); + + auto table_id = getStorageID(); + global_context.checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, part->getBytesOnDisk()); } void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String & name, bool moving_part, const Context & context) From d7de4509a3b2e26ca8f5f417ad4abb09818797fe Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Nov 2020 19:07:27 +0300 Subject: [PATCH 149/314] Update ActionsVisitor.cpp --- src/Interpreters/ActionsVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 3054f4781d0..67ef37ba319 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1049,7 +1049,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, * in the subquery_for_set object, this subquery is set as source and the temporary table _data1 as the table. * - this function shows the expression IN_data1. - * + * * In case that we have HAVING with IN subquery, we have to force creating set for it. * Also it doesn't make sence if it is GLOBAL IN or ordinary IN. */ From a9d4c0a4f97d9f73fab83d02d4a62909001e03fc Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Tue, 3 Nov 2020 19:59:45 +0300 Subject: [PATCH 150/314] open telementry log --- src/Interpreters/OpenTelemetrySpanLog.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/OpenTelemetrySpanLog.h b/src/Interpreters/OpenTelemetrySpanLog.h index fb382ee3177..271d02804f4 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.h +++ b/src/Interpreters/OpenTelemetrySpanLog.h @@ -11,8 +11,8 @@ struct OpenTelemetrySpan UInt64 span_id; UInt64 parent_span_id; std::string operation_name; - UInt64 start_time_us; - UInt64 finish_time_us; + Decimal64 start_time_us; + Decimal64 finish_time_us; UInt64 duration_ns; Array attribute_names; Array attribute_values; From e7f15dff557c2d267c1aa6d4ebfb3935e23536f9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 3 Nov 2020 21:22:46 +0300 Subject: [PATCH 151/314] Do not pass StoragePtr to ReadInOrderOptimizer::getInputOrder() Looks like this is not required anymore, since #11745 --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Storages/ReadInOrderOptimizer.cpp | 21 +++------------------ src/Storages/ReadInOrderOptimizer.h | 2 +- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageMaterializedView.cpp | 2 +- src/Storages/StorageMerge.cpp | 2 +- 6 files changed, 8 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d9821be4e4e..b376fc36ee9 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1453,7 +1453,7 @@ void InterpreterSelectQuery::executeFetchColumns( getSortDescriptionFromGroupBy(query), query_info.syntax_analyzer_result); - query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage, metadata_snapshot); + query_info.input_order_info = query_info.order_optimizer->getInputOrder(metadata_snapshot); } StreamLocalLimits limits; diff --git a/src/Storages/ReadInOrderOptimizer.cpp b/src/Storages/ReadInOrderOptimizer.cpp index 37f07ad1876..2bff5aee9dc 100644 --- a/src/Storages/ReadInOrderOptimizer.cpp +++ b/src/Storages/ReadInOrderOptimizer.cpp @@ -30,26 +30,11 @@ ReadInOrderOptimizer::ReadInOrderOptimizer( forbidden_columns.insert(elem.first); } -InputOrderInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot) const +InputOrderInfoPtr ReadInOrderOptimizer::getInputOrder(const StorageMetadataPtr & metadata_snapshot) const { - Names sorting_key_columns; - if (dynamic_cast(storage.get())) - { - if (!metadata_snapshot->hasSortingKey()) - return {}; - sorting_key_columns = metadata_snapshot->getSortingKeyColumns(); - } - else if (dynamic_cast(storage.get())) - { - if (!metadata_snapshot->hasSortingKey()) - return {}; - sorting_key_columns = metadata_snapshot->getSortingKeyColumns(); - } - else /// Inapplicable storage type - { + Names sorting_key_columns = metadata_snapshot->getSortingKeyColumns(); + if (!metadata_snapshot->hasSortingKey()) return {}; - } - SortDescription order_key_prefix_descr; int read_direction = required_sort_description.at(0).direction; diff --git a/src/Storages/ReadInOrderOptimizer.h b/src/Storages/ReadInOrderOptimizer.h index 3a16a10f89b..7a268189222 100644 --- a/src/Storages/ReadInOrderOptimizer.h +++ b/src/Storages/ReadInOrderOptimizer.h @@ -20,7 +20,7 @@ public: const SortDescription & required_sort_description, const TreeRewriterResultPtr & syntax_result); - InputOrderInfoPtr getInputOrder(const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot) const; + InputOrderInfoPtr getInputOrder(const StorageMetadataPtr & metadata_snapshot) const; private: /// Actions for every element of order expression to analyze functions for monotonicity diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index cf1eaa1f46d..9a24623d789 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -179,7 +179,7 @@ Pipe StorageBuffer::read( if (dst_has_same_structure) { if (query_info.order_optimizer) - query_info.input_order_info = query_info.order_optimizer->getInputOrder(destination, destination_metadata_snapshot); + query_info.input_order_info = query_info.order_optimizer->getInputOrder(destination_metadata_snapshot); /// The destination table has the same structure of the requested columns and we can simply read blocks from there. pipe_from_dst = destination->read( diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 8591bebe5cc..6287e2ad278 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -120,7 +120,7 @@ Pipe StorageMaterializedView::read( auto metadata_snapshot = storage->getInMemoryMetadataPtr(); if (query_info.order_optimizer) - query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage, metadata_snapshot); + query_info.input_order_info = query_info.order_optimizer->getInputOrder(metadata_snapshot); Pipe pipe = storage->read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); pipe.addTableLock(lock); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 1400689b990..12588654872 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -211,7 +211,7 @@ Pipe StorageMerge::read( { auto storage_ptr = std::get<0>(*it); auto storage_metadata_snapshot = storage_ptr->getInMemoryMetadataPtr(); - auto current_info = query_info.order_optimizer->getInputOrder(storage_ptr, storage_metadata_snapshot); + auto current_info = query_info.order_optimizer->getInputOrder(storage_metadata_snapshot); if (it == selected_tables.begin()) input_sorting_info = current_info; else if (!current_info || (input_sorting_info && *current_info != *input_sorting_info)) From aad09fc6f47c6fe182d9cc35f4a89f5f42ceea2d Mon Sep 17 00:00:00 2001 From: George Date: Tue, 3 Nov 2020 22:53:09 +0300 Subject: [PATCH 152/314] Added reinterpretAsUUID description --- docs/en/sql-reference/functions/type-conversion-functions.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index ba8d379ccec..bcd4d9c7016 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -323,6 +323,10 @@ This function accepts a number or date or date with time, and returns a string c This function accepts a number or date or date with time, and returns a FixedString containing bytes representing the corresponding value in host order (little endian). Null bytes are dropped from the end. For example, a UInt32 type value of 255 is a FixedString that is one byte long. +## reinterpretAsUUID {#reinterpretasuuid} + +This function accepts FixedString, and returns UUID. Takes 16 bytes string. If the string isn't long enough, the functions work as if the string is padded with the necessary number of null bytes to the rear. If the string longer than 16 bytes, the extra bytes in the rear are ignored. + ## CAST(x, T) {#type_conversion_function-cast} Converts ‘x’ to the ‘t’ data type. The syntax CAST(x AS t) is also supported. From 5fe679324efd8c657726b3de38fc6a36a3dbe2cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 3 Nov 2020 23:26:55 +0300 Subject: [PATCH 153/314] Improve performance of quantileMerge #16640 --- src/AggregateFunctions/ReservoirSampler.h | 30 +++++++++++++---------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/src/AggregateFunctions/ReservoirSampler.h b/src/AggregateFunctions/ReservoirSampler.h index 2ff5ab863af..594ff689aa5 100644 --- a/src/AggregateFunctions/ReservoirSampler.h +++ b/src/AggregateFunctions/ReservoirSampler.h @@ -158,12 +158,25 @@ public: } else { - randomShuffle(samples); + /// Replace every element in our reservoir to the b's reservoir + /// with the probability of b.total_values / (a.total_values + b.total_values) + /// Do it more roughly than true random sampling to save performance. + total_values += b.total_values; - for (size_t i = 0; i < sample_count; ++i) + + /// Will replace every frequency'th element in a to element from b. + double frequency = static_cast(total_values) / b.total_values; + + /// When frequency is too low, replace just one random element with the corresponding probability. + if (frequency * 2 >= sample_count) { - UInt64 rnd = genRandom(total_values); - if (rnd < b.total_values) + UInt64 rnd = genRandom(frequency); + if (rnd < sample_count) + samples[rnd] = b.samples[rnd]; + } + else + { + for (double i = 0; i < sample_count; i += frequency) samples[i] = b.samples[i]; } } @@ -222,15 +235,6 @@ private: return (static_cast(rng()) * (static_cast(rng.max()) + 1ULL) + static_cast(rng())) % lim; } - void randomShuffle(Array & v) - { - for (size_t i = 1; i < v.size(); ++i) - { - size_t j = genRandom(i + 1); - std::swap(v[i], v[j]); - } - } - void sortIfNeeded() { if (sorted) From fdd7f31cf7ef566c351410176f4fcf35181bc7e7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 3 Nov 2020 23:28:23 +0300 Subject: [PATCH 154/314] Add a test --- tests/performance/quantile_merge.xml | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 tests/performance/quantile_merge.xml diff --git a/tests/performance/quantile_merge.xml b/tests/performance/quantile_merge.xml new file mode 100644 index 00000000000..7f4d85a254c --- /dev/null +++ b/tests/performance/quantile_merge.xml @@ -0,0 +1,3 @@ + + SELECT quantileMerge(arrayJoin(arrayMap(x -> state, range(1000000)))) FROM (SELECT quantileState(rand()) AS state FROM numbers(10000)) + From 0b1c96db51e4b0a8f6f9aff7988434b1c7eb104d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 3 Nov 2020 23:32:18 +0300 Subject: [PATCH 155/314] Provide row number when "Cannot read all data" during parsing input --- src/Processors/Formats/IRowInputFormat.cpp | 2 ++ src/Processors/Formats/IRowInputFormat.h | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index 12d4db1f4a8..79df9f6956c 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -14,6 +14,7 @@ namespace ErrorCodes extern const int CANNOT_PARSE_DATE; extern const int CANNOT_PARSE_DATETIME; extern const int CANNOT_READ_ARRAY_FROM_TEXT; + extern const int CANNOT_READ_ALL_DATA; extern const int CANNOT_PARSE_NUMBER; extern const int CANNOT_PARSE_UUID; extern const int TOO_LARGE_STRING_SIZE; @@ -32,6 +33,7 @@ bool isParseError(int code) || code == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT || code == ErrorCodes::CANNOT_PARSE_NUMBER || code == ErrorCodes::CANNOT_PARSE_UUID + || code == ErrorCodes::CANNOT_READ_ALL_DATA || code == ErrorCodes::TOO_LARGE_STRING_SIZE || code == ErrorCodes::ARGUMENT_OUT_OF_BOUND /// For Decimals || code == ErrorCodes::INCORRECT_DATA; /// For some ReadHelpers diff --git a/src/Processors/Formats/IRowInputFormat.h b/src/Processors/Formats/IRowInputFormat.h index 1931fba2a0d..6571359bea5 100644 --- a/src/Processors/Formats/IRowInputFormat.h +++ b/src/Processors/Formats/IRowInputFormat.h @@ -37,7 +37,7 @@ struct RowInputFormatParams bool isParseError(int code); bool checkTimeLimit(const RowInputFormatParams & params, const Stopwatch & stopwatch); -///Row oriented input format: reads data row by row. +/// Row oriented input format: reads data row by row. class IRowInputFormat : public IInputFormat { public: From 213f290c09876e58940f65653dcb401861d65cfa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 3 Nov 2020 23:35:03 +0300 Subject: [PATCH 156/314] Add a test --- .../01554_row_number_after_cannot_read_all_data.reference | 1 + .../01554_row_number_after_cannot_read_all_data.sh | 6 ++++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/01554_row_number_after_cannot_read_all_data.reference create mode 100755 tests/queries/0_stateless/01554_row_number_after_cannot_read_all_data.sh diff --git a/tests/queries/0_stateless/01554_row_number_after_cannot_read_all_data.reference b/tests/queries/0_stateless/01554_row_number_after_cannot_read_all_data.reference new file mode 100644 index 00000000000..00bd5c46d6d --- /dev/null +++ b/tests/queries/0_stateless/01554_row_number_after_cannot_read_all_data.reference @@ -0,0 +1 @@ +(at row 2) diff --git a/tests/queries/0_stateless/01554_row_number_after_cannot_read_all_data.sh b/tests/queries/0_stateless/01554_row_number_after_cannot_read_all_data.sh new file mode 100755 index 00000000000..a29b44d2f16 --- /dev/null +++ b/tests/queries/0_stateless/01554_row_number_after_cannot_read_all_data.sh @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +echo -n -e '\x01\x00\x00\x00\x05Hello\x80' | ${CLICKHOUSE_LOCAL} --structure 'x UInt32, s String' --query "SELECT * FROM table" --input-format RowBinary 2>&1 | grep -oF '(at row 2)' From f5ed9adad303d94d03529587aae21913d6c69ea3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 3 Nov 2020 23:52:57 +0300 Subject: [PATCH 157/314] Remove old code --- src/AggregateFunctions/ReservoirSampler.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/AggregateFunctions/ReservoirSampler.h b/src/AggregateFunctions/ReservoirSampler.h index 2ff5ab863af..abf41ad6581 100644 --- a/src/AggregateFunctions/ReservoirSampler.h +++ b/src/AggregateFunctions/ReservoirSampler.h @@ -200,9 +200,6 @@ public: } private: - friend void qdigest_test(int normal_size, UInt64 value_limit, const std::vector & values, int queries_count, bool verbose); - friend void rs_perf_test(); - /// We allocate a little memory on the stack - to avoid allocations when there are many objects with a small number of elements. using Array = DB::PODArrayWithStackMemory; From 3ea2047d8f2aa93e4a4543751f19e0f4cd90afea Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 4 Nov 2020 01:46:52 +0300 Subject: [PATCH 158/314] Update index.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Добавил описание возможности использования строкового литерала. --- docs/en/sql-reference/operators/index.md | 28 +++++++++++++++++++++--- 1 file changed, 25 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index 262ae00dc95..274f7269bc8 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -151,21 +151,43 @@ Types of intervals: - `QUARTER` - `YEAR` +You can also use a string literal when setting the `INTERVAL` value. For example, `INTERVAL 1 HOUR` is identical to the `INTERVAL '1 hour'` or `INTERVAL '1' hour`. + !!! warning "Warning" Intervals with different types can’t be combined. You can’t use expressions like `INTERVAL 4 DAY 1 HOUR`. Specify intervals in units that are smaller or equal to the smallest unit of the interval, for example, `INTERVAL 25 HOUR`. You can use consecutive operations, like in the example below. -Example: +Examples: ``` sql -SELECT now() AS current_date_time, current_date_time + INTERVAL 4 DAY + INTERVAL 3 HOUR +SELECT now() AS current_date_time, current_date_time + INTERVAL 4 DAY + INTERVAL 3 HOUR; ``` ``` text ┌───current_date_time─┬─plus(plus(now(), toIntervalDay(4)), toIntervalHour(3))─┐ -│ 2019-10-23 11:16:28 │ 2019-10-27 14:16:28 │ +│ 2020-11-03 22:09:50 │ 2020-11-08 01:09:50 │ └─────────────────────┴────────────────────────────────────────────────────────┘ ``` +``` sql +SELECT now() AS current_date_time, current_date_time + INTERVAL '4 day' + INTERVAL '3 hour'; +``` + +``` text +┌───current_date_time─┬─plus(plus(now(), toIntervalDay(4)), toIntervalHour(3))─┐ +│ 2020-11-03 22:12:10 │ 2020-11-08 01:12:10 │ +└─────────────────────┴────────────────────────────────────────────────────────┘ +``` + +``` sql +SELECT now() AS current_date_time, current_date_time + INTERVAL '4' day + INTERVAL '3' hour; +``` + +``` text +┌───current_date_time─┬─plus(plus(now(), toIntervalDay('4')), toIntervalHour('3'))─┐ +│ 2020-11-03 22:33:19 │ 2020-11-08 01:33:19 │ +└─────────────────────┴────────────────────────────────────────────────────────────┘ +``` + **See Also** - [Interval](../../sql-reference/data-types/special-data-types/interval.md) data type From 6c37133f2c30bac50ad9ddc95d3c2db9c48f19d4 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 3 Nov 2020 20:05:06 -0400 Subject: [PATCH 159/314] Update uuid-functions.md Russian doc for toUUIDOrNull toUUIDOrZero. --- .../sql-reference/functions/uuid-functions.md | 48 +++++++++++++++++++ 1 file changed, 48 insertions(+) diff --git a/docs/ru/sql-reference/functions/uuid-functions.md b/docs/ru/sql-reference/functions/uuid-functions.md index 389ce751ce0..6082fcaa712 100644 --- a/docs/ru/sql-reference/functions/uuid-functions.md +++ b/docs/ru/sql-reference/functions/uuid-functions.md @@ -59,6 +59,54 @@ SELECT toUUID('61f0c404-5cb3-11e7-907b-a6006ad3dba0') AS uuid └──────────────────────────────────────┘ ``` +## toUUIDOrNull (x) {#touuidornull-x} + +Принимает строку, и пытается преобразовать в тип UUID. При неудаче возвращает NULL. + +``` sql +toUUIDOrNull(String) +``` + +**Возвращаемое значение** + +Значение типа Nullable(UUID). + +**Пример использования** + +``` sql +SELECT toUUIDOrNull('61f0c404-5cb3-11e7-907b-a6006ad3dba0T') AS uuid +``` + +``` text +┌─uuid─┐ +│ ᴺᵁᴸᴸ │ +└──────┘ +``` + +## toUUIDOrZero (x) {#touuidorzero-x} + +Принимает строку, и пытается преобразовать в тип UUID. При неудаче возвращает нулевой UUID. + +``` sql +toUUIDOrZero(String) +``` + +**Возвращаемое значение** + +Значение типа UUID. + +**Пример использования** + +``` sql +SELECT toUUIDOrZero('61f0c404-5cb3-11e7-907b-a6006ad3dba0T') AS uuid +``` + +``` text +┌─────────────────────────────────uuid─┐ +│ 00000000-0000-0000-0000-000000000000 │ +└──────────────────────────────────────┘ +``` + ## UUIDStringToNum {#uuidstringtonum} Принимает строку, содержащую 36 символов в формате `xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`, и возвращает в виде набора байт в [FixedString(16)](../../sql-reference/functions/uuid-functions.md). From 64bd63ca4999297e041fe853c6e4b6a2fa5bd5f0 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Wed, 4 Nov 2020 03:08:55 +0300 Subject: [PATCH 160/314] Try to parse DataType arguments as another nested type (#16262) * Try to parse DataType arguments as another nested one * Allow mixed lists of arguments of data types * Restore croaring back * Fix tests --- src/Parsers/ParserDataType.cpp | 31 +++++++++++++++++-- ...oom_filter_index_string_multi_granulas.sql | 3 ++ .../01532_tuple_with_name_type.reference | 5 +++ .../01532_tuple_with_name_type.sql | 21 +++++++++++++ 4 files changed, 57 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01532_tuple_with_name_type.reference create mode 100644 tests/queries/0_stateless/01532_tuple_with_name_type.sql diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index a0a4eb97efe..ee746329bff 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -1,13 +1,38 @@ #include -#include -#include + #include #include +#include +#include #include + namespace DB { +namespace +{ + +/// Wrapper to allow mixed lists of nested and normal types. +class ParserNestedTableOrExpression : public IParserBase +{ + private: + const char * getName() const override { return "data type or expression"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + ParserNestedTable parser1; + + if (parser1.parse(pos, node, expected)) + return true; + + ParserExpression parser2; + + return parser2.parse(pos, node, expected); + } +}; + +} + bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserNestedTable nested; @@ -78,7 +103,7 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ++pos; /// Parse optional parameters - ParserList args_parser(std::make_unique(), std::make_unique(TokenType::Comma)); + ParserList args_parser(std::make_unique(), std::make_unique(TokenType::Comma)); ASTPtr expr_list_args; if (!args_parser.parse(pos, expr_list_args, expected)) diff --git a/tests/queries/0_stateless/01307_bloom_filter_index_string_multi_granulas.sql b/tests/queries/0_stateless/01307_bloom_filter_index_string_multi_granulas.sql index 832f7140af2..e96c70bef7f 100644 --- a/tests/queries/0_stateless/01307_bloom_filter_index_string_multi_granulas.sql +++ b/tests/queries/0_stateless/01307_bloom_filter_index_string_multi_granulas.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS test_01307; + CREATE TABLE test_01307 (id UInt64, val String, INDEX ind val TYPE bloom_filter() GRANULARITY 1) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 2; INSERT INTO test_01307 (id, val) select number as id, toString(number) as val from numbers(4); SELECT count() FROM test_01307 WHERE identity(val) = '2'; @@ -6,3 +7,5 @@ SELECT count() FROM test_01307 WHERE val = '2'; OPTIMIZE TABLE test_01307 FINAL; SELECT count() FROM test_01307 WHERE identity(val) = '2'; SELECT count() FROM test_01307 WHERE val = '2'; + +DROP TABLE test_01307; diff --git a/tests/queries/0_stateless/01532_tuple_with_name_type.reference b/tests/queries/0_stateless/01532_tuple_with_name_type.reference new file mode 100644 index 00000000000..f9f6b5995ce --- /dev/null +++ b/tests/queries/0_stateless/01532_tuple_with_name_type.reference @@ -0,0 +1,5 @@ +a Tuple(key String, value String) +a Tuple(Tuple(key String, value String)) +a.key Array(String) +a.value Array(String) +a Tuple(UInt8, Tuple(key String, value String)) diff --git a/tests/queries/0_stateless/01532_tuple_with_name_type.sql b/tests/queries/0_stateless/01532_tuple_with_name_type.sql new file mode 100644 index 00000000000..fbc052d3cc0 --- /dev/null +++ b/tests/queries/0_stateless/01532_tuple_with_name_type.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS test_01532_1; +DROP TABLE IF EXISTS test_01532_2; +DROP TABLE IF EXISTS test_01532_3; +DROP TABLE IF EXISTS test_01532_4; + +CREATE TABLE test_01532_1 (a Tuple(key String, value String)) ENGINE Memory(); +DESCRIBE TABLE test_01532_1; + +CREATE TABLE test_01532_2 (a Tuple(Tuple(key String, value String))) ENGINE Memory(); +DESCRIBE TABLE test_01532_2; + +CREATE TABLE test_01532_3 (a Array(Tuple(key String, value String))) ENGINE Memory(); +DESCRIBE TABLE test_01532_3; + +CREATE TABLE test_01532_4 (a Tuple(UInt8, Tuple(key String, value String))) ENGINE Memory(); +DESCRIBE TABLE test_01532_4; + +DROP TABLE test_01532_1; +DROP TABLE test_01532_2; +DROP TABLE test_01532_3; +DROP TABLE test_01532_4; From dea3641e5c88815a9d448bcb32b2861159a9e9d5 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 3 Nov 2020 20:47:42 -0400 Subject: [PATCH 161/314] Update external-dicts-dict-layout.md --- .../external-dictionaries/external-dicts-dict-layout.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index ae2de0ba6f8..a49ddf62eb9 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -59,7 +59,8 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [range_hashed](#range-hashed) - [complex_key_hashed](#complex-key-hashed) - [complex_key_cache](#complex-key-cache) -- [ssd_complex_key_cache](#ssd-cache) +- [ssd_cache](#ssd-cache) +- [ssd_complex_key_cache](#complex-key-ssd-cache) - [complex_key_direct](#complex-key-direct) - [ip_trie](#ip-trie) From 52f3ef01d4a21ea23acfbd012e4a1bcf4c694b69 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 3 Nov 2020 20:55:54 -0400 Subject: [PATCH 162/314] Update external-dicts-dict-layout.md --- .../external-dicts-dict-layout.md | 34 +++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 979f1f2e5b9..70a99dd6841 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -299,6 +299,40 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) Тип размещения предназначен для использования с составными [ключами](external-dicts-dict-structure.md). Аналогичен `cache`. +### ssd_cache {#ssd-cache} + +Похож на `cache`, но хранит данные на SSD и индекс в оперативной памяти. + +``` xml + + + + 4096 + + 16777216 + + 131072 + + 1048576 + + /var/lib/clickhouse/clickhouse_dictionaries/test_dict + + 1048576 + + +``` + +или + +``` sql +LAYOUT(CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576 + PATH /var/lib/clickhouse/clickhouse_dictionaries/test_dict MAX_STORED_KEYS 1048576)) +``` + +### complex_key_ssd_cache {#complex-key-ssd-cache} + +Тип размещения предназначен для использования с составными [ключами](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Похож на `ssd_cache`. + ### direct {#direct} Словарь не хранит данные локально и взаимодействует с источником непосредственно в момент запроса. From e945e256bbbe124918fea28831a0c3b81b575e3d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 3 Nov 2020 20:57:14 -0400 Subject: [PATCH 163/314] Update external-dicts-dict-layout.md --- .../external-dictionaries/external-dicts-dict-layout.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 70a99dd6841..e97b1e421a4 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -54,6 +54,8 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [hashed](#dicts-external_dicts_dict_layout-hashed) - [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) - [cache](#cache) +- [ssd_cache](#ssd-cache) +- [ssd_complex_key_cache](#complex-key-ssd-cache) - [direct](#direct) - [range_hashed](#range-hashed) - [complex_key_hashed](#complex-key-hashed) From 5cb708a50ab8025bf7110d4c64b38a8494c62e44 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 4 Nov 2020 04:19:53 +0300 Subject: [PATCH 164/314] fixup --- docker/test/performance-comparison/compare.sh | 74 +++++++++---------- 1 file changed, 35 insertions(+), 39 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index ca52fbd5891..08462298093 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1083,45 +1083,41 @@ function upload_results fi # Surprisingly, clickhouse-client doesn't understand --host 127.0.0.1:9000 - # so I have to do this instead. I tried to use Poco URI parser for this, - # but it's also broken and can't parse host:port. - IFS=':' read -r host port <<<"${CHPC_DATABASE_URL}" - - upload_client=(clickhouse-client - --host "${host}" - --port "${port}" - --secure - --user "${CHPC_DATABASE_USER}" - --password "${CHPC_DATABASE_PASSWORD}" - --config "ch/tests/config/client_config.xml" - --database perftest - -m - "--date_time_input_format=best_effort") - - set +x # Don't show password in the log - "${upload_client[@]}" --query " - insert into query_metrics_tmp - select - toDate(event_time) event_date, - toDateTime('$(cd right/ch && git show -s --format=%ci "$SHA_TO_TEST" | cut -d' ' -f-2)') event_time, - $PR_TO_TEST pr_number, - '$REF_SHA' old_sha, - '$SHA_TO_TEST' new_sha, - test, - query_index, - query_display_name, - metric_name, - old_value, - new_value, - diff, - stat_threshold - from input('metric_name text, old_value float, new_value float, diff float, - ratio_display_text text, stat_threshold float, - test text, query_index int, query_display_name text') - settings date_time_input_format='best_effort' - format TSV - settings date_time_input_format='best_effort' - " < "report/all-query-metrics.tsv" + # so I have to extract host and port with clickhouse-local. I tried to use + # Poco URI parser to support this in the client, but it's broken and can't + # parse host:port. + set +x # Don't show password in the log + clickhouse-client \ + $(clickhouse-local --query "with '${CHPC_DATABASE_URL}' as url select '--host ' || domain(url) || ' --port ' || toString(port(url)) format TSV") \ + --secure \ + --user "${CHPC_DATABASE_USER}" \ + --password "${CHPC_DATABASE_PASSWORD}" \ + --config "ch/tests/config/client_config.xml" \ + --database perftest \ + --date_time_input_format=best_effort \ + --query " + insert into query_metrics_tmp + select + toDate(event_time) event_date, + toDateTime('$(cd right/ch && git show -s --format=%ci "$SHA_TO_TEST" | cut -d' ' -f-2)') event_time, + $PR_TO_TEST pr_number, + '$REF_SHA' old_sha, + '$SHA_TO_TEST' new_sha, + test, + query_index, + query_display_name, + metric_name, + old_value, + new_value, + diff, + stat_threshold + from input('metric_name text, old_value float, new_value float, diff float, + ratio_display_text text, stat_threshold float, + test text, query_index int, query_display_name text') + settings date_time_input_format='best_effort' + format TSV + settings date_time_input_format='best_effort' +" < report/all-query-metrics.tsv # Don't leave whitespace after INSERT: https://github.com/ClickHouse/ClickHouse/issues/16652 set -x } From b161127fc1223c40d5225f7deac594dc4bc95179 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 3 Nov 2020 21:19:50 +0300 Subject: [PATCH 165/314] Add a test for spreading parts between threads for ReadInOrderOptimizer --- ...1551_mergetree_read_in_order_spread.reference | 11 +++++++++++ .../01551_mergetree_read_in_order_spread.sql | 16 ++++++++++++++++ 2 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference create mode 100644 tests/queries/0_stateless/01551_mergetree_read_in_order_spread.sql diff --git a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference new file mode 100644 index 00000000000..fc10b4707a9 --- /dev/null +++ b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference @@ -0,0 +1,11 @@ +(Expression) +ExpressionTransform + (Expression) + ExpressionTransform + (Aggregating) + FinalizingSimpleTransform + AggregatingSortedTransform 3 → 1 + AggregatingInOrderTransform × 3 + (Expression) + ExpressionTransform × 3 + (ReadFromStorage) diff --git a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.sql b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.sql new file mode 100644 index 00000000000..831a7282861 --- /dev/null +++ b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS data_01551; + +CREATE TABLE data_01551 +( + key UInt32 +) engine=AggregatingMergeTree() +PARTITION BY key%2 +ORDER BY (key, key/2) +SETTINGS index_granularity=10; + +INSERT INTO data_01551 SELECT number FROM numbers(100000); +SET max_threads=3; +SET merge_tree_min_rows_for_concurrent_read=10000; +SET optimize_aggregation_in_order=1; +SET read_in_order_two_level_merge_threshold=1; +EXPLAIN PIPELINE SELECT key FROM data_01551 GROUP BY key, key/2; From 2389406c21848d07da7f2fc670a24612c018f6e4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 3 Nov 2020 21:19:50 +0300 Subject: [PATCH 166/314] Fix spreading for ReadInOrderOptimizer with expression in ORDER BY This will fix optimize_read_in_order/optimize_aggregation_in_order with max_threads>0 and expression in ORDER BY --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 56 ++++++++++++++----- 1 file changed, 43 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index f06bfb97b2c..a38d50e56fb 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -82,6 +82,17 @@ static Block getBlockWithPartColumn(const MergeTreeData::DataPartsVector & parts return Block{ColumnWithTypeAndName(std::move(column), std::make_shared(), "_part")}; } +/// Check if ORDER BY clause of the query has some expression. +static bool sortingDescriptionHasExpressions(const SortDescription & sort_description, const StorageMetadataPtr & metadata_snapshot) +{ + auto all_columns = metadata_snapshot->getColumns(); + for (const auto & sort_column : sort_description) + { + if (!all_columns.has(sort_column.column_name)) + return true; + } + return false; +} size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead( const MergeTreeData::DataPartsVector & parts, @@ -1065,6 +1076,7 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( const size_t min_marks_per_stream = (sum_marks - 1) / num_streams + 1; bool need_preliminary_merge = (parts.size() > settings.read_in_order_two_level_merge_threshold); + size_t max_output_ports = 0; for (size_t i = 0; i < num_streams && !parts.empty(); ++i) { @@ -1174,25 +1186,43 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( }); } - if (pipe.numOutputPorts() > 1 && need_preliminary_merge) + max_output_ports = std::max(pipe.numOutputPorts(), max_output_ports); + res.emplace_back(std::move(pipe)); + } + + if (need_preliminary_merge) + { + /// If ORDER BY clause of the query contains some expression, + /// then those new columns should be added for the merge step, + /// and this should be done always, if there is at least one pipe that + /// has multiple output ports. + bool sorting_key_has_expression = sortingDescriptionHasExpressions(input_order_info->order_key_prefix_descr, metadata_snapshot); + bool force_sorting_key_transform = res.size() > 1 && max_output_ports > 1 && sorting_key_has_expression; + + for (auto & pipe : res) { SortDescription sort_description; - for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j) - sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j], - input_order_info->direction, 1); - /// Drop temporary columns, added by 'sorting_key_prefix_expr' - out_projection = createProjection(pipe, data); - pipe.addSimpleTransform([sorting_key_prefix_expr](const Block & header) + if (pipe.numOutputPorts() > 1 || force_sorting_key_transform) { - return std::make_shared(header, sorting_key_prefix_expr); - }); + for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j) + sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j], + input_order_info->direction, 1); - pipe.addTransform(std::make_shared( - pipe.getHeader(), pipe.numOutputPorts(), sort_description, max_block_size)); + /// Drop temporary columns, added by 'sorting_key_prefix_expr' + out_projection = createProjection(pipe, data); + pipe.addSimpleTransform([sorting_key_prefix_expr](const Block & header) + { + return std::make_shared(header, sorting_key_prefix_expr); + }); + } + + if (pipe.numOutputPorts() > 1) + { + pipe.addTransform(std::make_shared( + pipe.getHeader(), pipe.numOutputPorts(), sort_description, max_block_size)); + } } - - res.emplace_back(std::move(pipe)); } return Pipe::unitePipes(std::move(res)); From 26b1024feaa7a48a94c9129c5300fb5a505bd2c1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 4 Nov 2020 07:12:29 +0300 Subject: [PATCH 167/314] Fix 01339_client_unrecognized_option after UNRECOGNIZED_ARGUMENTS value change --- tests/queries/0_stateless/01339_client_unrecognized_option.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01339_client_unrecognized_option.sh b/tests/queries/0_stateless/01339_client_unrecognized_option.sh index 09e3a2b291f..13c286cd032 100755 --- a/tests/queries/0_stateless/01339_client_unrecognized_option.sh +++ b/tests/queries/0_stateless/01339_client_unrecognized_option.sh @@ -3,13 +3,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT xyzgarbage 2>&1 | grep -q "Code: 2003" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT xyzgarbage 2>&1 | grep -q "Code: 552" && echo 'OK' || echo 'FAIL' $CLICKHOUSE_CLIENT -xyzgarbage 2>&1 | grep -q "Bad arguments" && echo 'OK' || echo 'FAIL' $CLICKHOUSE_CLIENT --xyzgarbage 2>&1 | grep -q "Bad arguments" && echo 'OK' || echo 'FAIL' -cat /etc/passwd | sed 's/:/\t/g' | $CLICKHOUSE_CLIENT --query="SELECT shell, count() AS c FROM passwd GROUP BY shell ORDER BY c DESC" --external --file=- --name=passwd --structure='login String, unused String, uid UInt16, gid UInt16, comment String, home String, shell String' xyzgarbage 2>&1 | grep -q "Code: 2003" && echo 'OK' || echo 'FAIL' +cat /etc/passwd | sed 's/:/\t/g' | $CLICKHOUSE_CLIENT --query="SELECT shell, count() AS c FROM passwd GROUP BY shell ORDER BY c DESC" --external --file=- --name=passwd --structure='login String, unused String, uid UInt16, gid UInt16, comment String, home String, shell String' xyzgarbage 2>&1 | grep -q "Code: 552" && echo 'OK' || echo 'FAIL' cat /etc/passwd | sed 's/:/\t/g' | $CLICKHOUSE_CLIENT --query="SELECT shell, count() AS c FROM passwd GROUP BY shell ORDER BY c DESC" --external -xyzgarbage --file=- --name=passwd --structure='login String, unused String, uid UInt16, gid UInt16, comment String, home String, shell String' 2>&1 | grep -q "Bad arguments" && echo 'OK' || echo 'FAIL' From 130263332a865771486ecc5b6fe43ad652d51513 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 10:52:42 +0300 Subject: [PATCH 168/314] Added results from Huawei --- website/benchmark/hardware/index.html | 1 + .../results/huawei_taishan_2280_3.json | 54 +++++++++++++++++++ 2 files changed, 55 insertions(+) create mode 100644 website/benchmark/hardware/results/huawei_taishan_2280_3.json diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index 68bc4b29653..73503179bc0 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -53,6 +53,7 @@ Results for Xeon 2176G are from Sergey Golod.
Results for Azure DS3v2 are from Boris Granveaud.
Results for AWS are from Wolf Kreuzerkrieg.
Results for Huawei Taishan are from Peng Gao in sina.com.
+Results for Huawei Taishan (2) are from Kurmaev Roman at Huawei.
Results for Selectel and AMD EPYC 7402P are from Andrey Dudin.
Results for ProLiant are from Denis Ustinov.
Results for AMD EPYC 7502P 128GiB are from Kostiantyn Velychkovskyi.
diff --git a/website/benchmark/hardware/results/huawei_taishan_2280_3.json b/website/benchmark/hardware/results/huawei_taishan_2280_3.json new file mode 100644 index 00000000000..c31b055dbee --- /dev/null +++ b/website/benchmark/hardware/results/huawei_taishan_2280_3.json @@ -0,0 +1,54 @@ +[ + { + "system": "Huawei TaiShan (2)", + "system_full": "Huawei TaiShan 2280 (AArch64) 2280 (2x64 cores, SSD)", + "time": "2020-11-03 00:00:00", + "kind": "server", + "result": + [ + [0.001, 0.003, 0.002], + [0.009, 0.024, 0.032], + [0.023, 0.051, 0.027], + [0.035, 0.063, 0.027], + [0.103, 0.145, 0.096], + [0.121, 0.191, 0.109], + [0.009, 0.026, 0.023], + [0.014, 0.025, 0.022], + [0.209, 0.157, 0.119], + [0.254, 0.182, 0.132], + [0.153, 0.101, 0.078], + [0.151, 0.099, 0.097], + [0.289, 0.238, 0.137], + [0.342, 0.255, 0.164], + [0.302, 0.216, 0.142], + [0.291, 0.202, 0.163], + [1.269, 0.483, 0.309], + [0.457, 0.313, 0.229], + [1.376, 0.942, 0.597], + [0.073, 0.055, 0.023], + [0.555, 0.321, 0.193], + [0.913, 0.308, 0.191], + [1.713, 0.668, 0.421], + [0.894, 0.837, 0.387], + [0.238, 0.091, 0.061], + [0.153, 0.081, 0.055], + [0.174, 0.091, 0.059], + [0.933, 0.361, 0.233], + [0.793, 0.366, 0.237], + [0.682, 0.551, 0.549], + [0.321, 0.213, 0.154], + [0.593, 0.293, 0.173], + [4.436, 1.481, 1.003], + [1.544, 0.997, 0.774], + [1.981, 1.006, 0.841], + [0.539, 0.339, 0.247], + [0.194, 0.186, 0.141], + [0.075, 0.072, 0.066], + [0.072, 0.087, 0.053], + [0.393, 0.398, 0.356], + [0.032, 0.042, 0.031], + [0.023, 0.023, 0.022], + [0.005, 0.006, 0.006] + ] + } +] From cd711039fb6a9a4d382801e089281a0dba9115ae Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 4 Nov 2020 07:56:27 +0000 Subject: [PATCH 169/314] Allow reinterpret between integers and floats of the same size --- .../registerFunctionsReinterpret.cpp | 4 +- ...nterpretStringAs.cpp => reinterpretAs.cpp} | 108 +++++++++++++----- src/Functions/ya.make | 2 +- 3 files changed, 84 insertions(+), 30 deletions(-) rename src/Functions/{reinterpretStringAs.cpp => reinterpretAs.cpp} (54%) diff --git a/src/Functions/registerFunctionsReinterpret.cpp b/src/Functions/registerFunctionsReinterpret.cpp index d2e43fbd52a..d82274ce9ed 100644 --- a/src/Functions/registerFunctionsReinterpret.cpp +++ b/src/Functions/registerFunctionsReinterpret.cpp @@ -3,13 +3,13 @@ namespace DB class FunctionFactory; -void registerFunctionsReinterpretStringAs(FunctionFactory & factory); +void registerFunctionsReinterpretAs(FunctionFactory & factory); void registerFunctionReinterpretAsString(FunctionFactory & factory); void registerFunctionReinterpretAsFixedString(FunctionFactory & factory); void registerFunctionsReinterpret(FunctionFactory & factory) { - registerFunctionsReinterpretStringAs(factory); + registerFunctionsReinterpretAs(factory); registerFunctionReinterpretAsString(factory); registerFunctionReinterpretAsFixedString(factory); } diff --git a/src/Functions/reinterpretStringAs.cpp b/src/Functions/reinterpretAs.cpp similarity index 54% rename from src/Functions/reinterpretStringAs.cpp rename to src/Functions/reinterpretAs.cpp index cd36d63dd46..1efadd4f491 100644 --- a/src/Functions/reinterpretStringAs.cpp +++ b/src/Functions/reinterpretAs.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -25,30 +26,60 @@ namespace ErrorCodes namespace { - -template -class FunctionReinterpretStringAs : public IFunction +template +class FunctionReinterpretAs : public IFunction { + template + static bool castType(const IDataType * type, F && f) + { + return castTypeToEither( + type, std::forward(f)); + } + + template + static void reinterpretImpl(const PaddedPODArray & from, PaddedPODArray & to) + { + const auto * from_reinterpret = reinterpret_cast(const_cast(from.data())); + to.resize(from.size()); + for (size_t i = 0; i < from.size(); ++i) + { + to[i] = from_reinterpret[i]; + } + } + public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(const Context &) { return std::make_shared(); } using ToFieldType = typename ToDataType::FieldType; using ColumnType = typename ToDataType::ColumnType; - String getName() const override - { - return name; - } + String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { const IDataType & type = *arguments[0]; - if (!isStringOrFixedString(type)) - throw Exception("Cannot reinterpret " + type.getName() + " as " + ToDataType().getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if constexpr (support_between_float_integer) + { + if (!isStringOrFixedString(type) && !isNumber(type)) + throw Exception( + "Cannot reinterpret " + type.getName() + " as " + ToDataType().getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (isNumber(type)) + { + if (type.getSizeOfValueInMemory() != ToDataType{}.getSizeOfValueInMemory()) + throw Exception( + "Cannot reinterpret " + type.getName() + " as " + ToDataType().getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + } + else + { + if (!isStringOrFixedString(type)) + throw Exception( + "Cannot reinterpret " + type.getName() + " as " + ToDataType().getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } return std::make_shared(); } @@ -99,10 +130,34 @@ public: return col_res; } + else if constexpr (support_between_float_integer) + { + ColumnPtr res; + if (castType(arguments[0].type.get(), [&](const auto & type) { + using DataType = std::decay_t; + using T = typename DataType::FieldType; + + const ColumnVector * col = checkAndGetColumn>(arguments[0].column.get()); + auto col_res = ColumnType::create(); + reinterpretImpl(col->getData(), col_res->getData()); + res = std::move(col_res); + + return true; + })) + { + return res; + } + else + { + throw Exception( + "Illegal column " + arguments[0].column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } + } else { - throw Exception("Illegal column " + arguments[0].column->getName() - + " of argument of function " + getName(), + throw Exception( + "Illegal column " + arguments[0].column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } } @@ -123,23 +178,22 @@ struct NameReinterpretAsDate { static constexpr auto name = "reinterpretA struct NameReinterpretAsDateTime { static constexpr auto name = "reinterpretAsDateTime"; }; struct NameReinterpretAsUUID { static constexpr auto name = "reinterpretAsUUID"; }; -using FunctionReinterpretAsUInt8 = FunctionReinterpretStringAs; -using FunctionReinterpretAsUInt16 = FunctionReinterpretStringAs; -using FunctionReinterpretAsUInt32 = FunctionReinterpretStringAs; -using FunctionReinterpretAsUInt64 = FunctionReinterpretStringAs; -using FunctionReinterpretAsInt8 = FunctionReinterpretStringAs; -using FunctionReinterpretAsInt16 = FunctionReinterpretStringAs; -using FunctionReinterpretAsInt32 = FunctionReinterpretStringAs; -using FunctionReinterpretAsInt64 = FunctionReinterpretStringAs; -using FunctionReinterpretAsFloat32 = FunctionReinterpretStringAs; -using FunctionReinterpretAsFloat64 = FunctionReinterpretStringAs; -using FunctionReinterpretAsDate = FunctionReinterpretStringAs; -using FunctionReinterpretAsDateTime = FunctionReinterpretStringAs; -using FunctionReinterpretAsUUID = FunctionReinterpretStringAs; - +using FunctionReinterpretAsUInt8 = FunctionReinterpretAs; +using FunctionReinterpretAsUInt16 = FunctionReinterpretAs; +using FunctionReinterpretAsUInt32 = FunctionReinterpretAs; +using FunctionReinterpretAsUInt64 = FunctionReinterpretAs; +using FunctionReinterpretAsInt8 = FunctionReinterpretAs; +using FunctionReinterpretAsInt16 = FunctionReinterpretAs; +using FunctionReinterpretAsInt32 = FunctionReinterpretAs; +using FunctionReinterpretAsInt64 = FunctionReinterpretAs; +using FunctionReinterpretAsFloat32 = FunctionReinterpretAs; +using FunctionReinterpretAsFloat64 = FunctionReinterpretAs; +using FunctionReinterpretAsDate = FunctionReinterpretAs; +using FunctionReinterpretAsDateTime = FunctionReinterpretAs; +using FunctionReinterpretAsUUID = FunctionReinterpretAs; } -void registerFunctionsReinterpretStringAs(FunctionFactory & factory) +void registerFunctionsReinterpretAs(FunctionFactory & factory) { factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 4c2cbaf5b1f..bc5af88b6af 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -359,7 +359,7 @@ SRCS( registerFunctionsVisitParam.cpp reinterpretAsFixedString.cpp reinterpretAsString.cpp - reinterpretStringAs.cpp + reinterpretAs.cpp repeat.cpp replaceAll.cpp replaceOne.cpp From b127752102594baab9ae9bc91a182b90adf76a59 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 4 Nov 2020 07:58:26 +0000 Subject: [PATCH 170/314] add test --- .../0_stateless/01554_interpreter_integer_float.reference | 8 ++++++++ .../0_stateless/01554_interpreter_integer_float.sql | 8 ++++++++ 2 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/01554_interpreter_integer_float.reference create mode 100644 tests/queries/0_stateless/01554_interpreter_integer_float.sql diff --git a/tests/queries/0_stateless/01554_interpreter_integer_float.reference b/tests/queries/0_stateless/01554_interpreter_integer_float.reference new file mode 100644 index 00000000000..0aab8e60c7b --- /dev/null +++ b/tests/queries/0_stateless/01554_interpreter_integer_float.reference @@ -0,0 +1,8 @@ +1.72999e-40 +1067320848 +1.72999e-40 +1067320848 +6.09954e-319 +4608238783128613432 +6.09954e-319 +4608238783128613432 diff --git a/tests/queries/0_stateless/01554_interpreter_integer_float.sql b/tests/queries/0_stateless/01554_interpreter_integer_float.sql new file mode 100644 index 00000000000..69b00fc68d4 --- /dev/null +++ b/tests/queries/0_stateless/01554_interpreter_integer_float.sql @@ -0,0 +1,8 @@ +SELECT reinterpretAsFloat32(CAST(123456 AS UInt32)); +SELECT reinterpretAsUInt32(CAST(1.23456 AS Float32)); +SELECT reinterpretAsFloat32(CAST(123456 AS Int32)); +SELECT reinterpretAsInt32(CAST(1.23456 AS Float32)); +SELECT reinterpretAsFloat64(CAST(123456 AS UInt64)); +SELECT reinterpretAsUInt64(CAST(1.23456 AS Float64)); +SELECT reinterpretAsFloat64(CAST(123456 AS Int64)); +SELECT reinterpretAsInt64(CAST(1.23456 AS Float64)); From 96d66a480fe2f532248ee283ae378ee1ed757920 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 4 Nov 2020 11:18:50 +0300 Subject: [PATCH 171/314] Update type-conversion-functions.md --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index bcd4d9c7016..4e1b7ebef02 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -325,7 +325,7 @@ This function accepts a number or date or date with time, and returns a FixedStr ## reinterpretAsUUID {#reinterpretasuuid} -This function accepts FixedString, and returns UUID. Takes 16 bytes string. If the string isn't long enough, the functions work as if the string is padded with the necessary number of null bytes to the rear. If the string longer than 16 bytes, the extra bytes in the rear are ignored. +This function accepts FixedString, and returns UUID. Takes 16 bytes string. If the string isn't long enough, the functions work as if the string is padded with the necessary number of null bytes to the end. If the string longer than 16 bytes, the extra bytes in the end are ignored. ## CAST(x, T) {#type_conversion_function-cast} From 788418b447c5b4de28bad4de397d0de77c89f781 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 4 Nov 2020 11:19:17 +0300 Subject: [PATCH 172/314] Update type-conversion-functions.md --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 4e1b7ebef02..5c8dc5fd272 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -325,7 +325,7 @@ This function accepts a number or date or date with time, and returns a FixedStr ## reinterpretAsUUID {#reinterpretasuuid} -This function accepts FixedString, and returns UUID. Takes 16 bytes string. If the string isn't long enough, the functions work as if the string is padded with the necessary number of null bytes to the end. If the string longer than 16 bytes, the extra bytes in the end are ignored. +This function accepts FixedString, and returns UUID. Takes 16 bytes string. If the string isn't long enough, the functions work as if the string is padded with the necessary number of null bytes to the end. If the string longer than 16 bytes, the extra bytes at the end are ignored. ## CAST(x, T) {#type_conversion_function-cast} From 0001fc244ddf3e083c5f77e94ce6a7c1bd6f281d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 4 Nov 2020 11:24:40 +0300 Subject: [PATCH 173/314] Update settings.md --- .../server-configuration-parameters/settings.md | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index dbece2d0cee..31a8e896438 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -481,9 +481,13 @@ The maximum number of simultaneously processed requests. ## max_concurrent_queries_for_all_users {#max-concurrent-queries-for-all-users} -The maximum number of simultaneously processed requests for all users. +Throw exception if the value of this setting is less or equal than the current number of simultaneously processed queries. -Default value: `0`. +Example: `max_concurrent_queries_for_all_users` can be set to 99 for all users and database administrator can set it to 100 for itself to run queries for investigation even when the server is overloaded. + +Modifying the setting for one query or user does not affect other queries. + +Default value: `0` that means no limit. **Example** @@ -491,8 +495,6 @@ Default value: `0`. 99 ``` -`max_concurrent_queries_for_all_users` can be set to 99 for all users and database administrator can set it to 100 for itself to run queries for investigation even when the server is overloaded. - **See Also** - [max_concurrent_queries](#max-concurrent-queries) From 74ebe952d1c712dcaa7cc1cab474a009e27e4672 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 4 Nov 2020 11:32:20 +0300 Subject: [PATCH 174/314] Update Context.cpp --- src/Interpreters/Context.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index be701ff171b..20b976b71dc 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -363,7 +363,7 @@ struct ContextShared /// Initialized on demand (on distributed storages initialization) since Settings should be initialized std::unique_ptr clusters; ConfigurationPtr clusters_config; /// Stores updated configs - ConfigurationPtr zookeeper_config; /// Stores zookeeperd configs + ConfigurationPtr zookeeper_config; /// Stores zookeeper configs mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config #if USE_EMBEDDED_COMPILER From 9b8dbf37cc336d8d59c6b8fd5b5faf2252191686 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 4 Nov 2020 08:30:09 +0000 Subject: [PATCH 175/314] fix fix --- src/Functions/reinterpretAs.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Functions/reinterpretAs.cpp b/src/Functions/reinterpretAs.cpp index 1efadd4f491..76ee6155048 100644 --- a/src/Functions/reinterpretAs.cpp +++ b/src/Functions/reinterpretAs.cpp @@ -15,6 +15,7 @@ #include #include +#include namespace DB { @@ -39,11 +40,10 @@ class FunctionReinterpretAs : public IFunction template static void reinterpretImpl(const PaddedPODArray & from, PaddedPODArray & to) { - const auto * from_reinterpret = reinterpret_cast(const_cast(from.data())); to.resize(from.size()); for (size_t i = 0; i < from.size(); ++i) { - to[i] = from_reinterpret[i]; + to[i] = unalignedLoad(&(from.data()[i])); } } @@ -133,7 +133,8 @@ public: else if constexpr (support_between_float_integer) { ColumnPtr res; - if (castType(arguments[0].type.get(), [&](const auto & type) { + if (castType(arguments[0].type.get(), [&](const auto & type) + { using DataType = std::decay_t; using T = typename DataType::FieldType; From c82d8dde3dad814db62b52d727e3732e62e2b2bf Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 4 Nov 2020 12:11:43 +0300 Subject: [PATCH 176/314] Fix inconsistent whitespace. --- src/Interpreters/ApplyWithSubqueryVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ApplyWithSubqueryVisitor.cpp b/src/Interpreters/ApplyWithSubqueryVisitor.cpp index d8ddbd2c2fa..c360a9683c3 100644 --- a/src/Interpreters/ApplyWithSubqueryVisitor.cpp +++ b/src/Interpreters/ApplyWithSubqueryVisitor.cpp @@ -18,7 +18,7 @@ void ApplyWithSubqueryVisitor::visit(ASTPtr & ast, const Data & data) { for (auto & child : with->children) { - visit(child, new_data ? *new_data: data); + visit(child, new_data ? *new_data : data); if (auto * ast_with_elem = child->as()) { if (!new_data) From cf61c29a03bb4ae7d29c0e7d4524be6f99695057 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 4 Nov 2020 12:17:53 +0300 Subject: [PATCH 177/314] Update docs/en/sql-reference/table-functions/null.md Co-authored-by: BayoNet --- docs/en/sql-reference/table-functions/null.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/null.md b/docs/en/sql-reference/table-functions/null.md index 988ba18d709..cc0fd8f9427 100644 --- a/docs/en/sql-reference/table-functions/null.md +++ b/docs/en/sql-reference/table-functions/null.md @@ -5,7 +5,7 @@ toc_title: null function # null {#null-function} -Accepts an inserted data of the specified structure and immediately drops it away. The function is used for convenience writing tests and demos. +Accepts an inserted data of the specified structure and immediately drops it away. The function is used for convenience writing tests and demonstrations. **Syntax** From 1f323e98d1bca559bbab974cb084b8bae7737552 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 4 Nov 2020 12:20:06 +0300 Subject: [PATCH 178/314] Update null.md --- docs/en/sql-reference/table-functions/null.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/table-functions/null.md b/docs/en/sql-reference/table-functions/null.md index cc0fd8f9427..6edec61add7 100644 --- a/docs/en/sql-reference/table-functions/null.md +++ b/docs/en/sql-reference/table-functions/null.md @@ -36,4 +36,6 @@ INSERT INTO t SELECT * FROM numbers_mt(1000000000); DROP TABLE IF EXISTS t; ``` +See also: format **Null**. + [Original article](https://clickhouse.tech/docs/en/sql-reference/table-functions/null/) From d7b4e036d2ef94b4316cf3ff688f36ba25bbf7cd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 12:27:34 +0300 Subject: [PATCH 179/314] Fix flaky tests after "max_memory_usage_for_user" --- .../01541_max_memory_usage_for_user.sh | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01541_max_memory_usage_for_user.sh b/tests/queries/0_stateless/01541_max_memory_usage_for_user.sh index ecae442b134..3b1ced03473 100755 --- a/tests/queries/0_stateless/01541_max_memory_usage_for_user.sh +++ b/tests/queries/0_stateless/01541_max_memory_usage_for_user.sh @@ -9,26 +9,30 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # - max_memory_usage_for_user # - one users' query in background (to avoid reseting max_memory_usage_for_user) -query="SELECT groupArray(repeat('a', 1000)) FROM numbers(10000) GROUP BY number%10 FORMAT JSON" +# We have to create a separate user to run this tests isolated from other tests. +${CLICKHOUSE_CLIENT} -n --query "DROP USER IF EXISTS test_01541; CREATE USER test_01541; GRANT ALL ON *.* TO test_01541;"; + + +query="SELECT groupArray(repeat('a', 1000)) FROM numbers(10000) GROUP BY number % 10 FORMAT JSON" function execute_http() { for _ in {1..100}; do - $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&max_memory_usage_for_user=100Mi&max_threads=1" -d@- <<<"$query" | grep -F DB::Exception: + $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&user=test_01541&max_memory_usage_for_user=100Mi&max_threads=1" -d@- <<<"$query" | grep -F DB::Exception: done } function execute_tcp() { # slow in debug, but should trigger the problem in ~10 iterations, so 20 is ok for _ in {1..20}; do - ${CLICKHOUSE_CLIENT} --max_memory_usage_for_user=100Mi --max_threads=1 -q "$query" | grep -F DB::Exception: + ${CLICKHOUSE_CLIENT} --user=test_01541 --max_memory_usage_for_user=100Mi --max_threads=1 -q "$query" | grep -F DB::Exception: done } function execute_tcp_one_session() { for _ in {1..30}; do echo "$query;" - done | ${CLICKHOUSE_CLIENT} -nm --max_memory_usage_for_user=100Mi --max_threads=1 | grep -F DB::Exception: + done | ${CLICKHOUSE_CLIENT} --user=test_01541 -nm --max_memory_usage_for_user=100Mi --max_threads=1 | grep -F DB::Exception: } @@ -36,7 +40,7 @@ function execute_tcp_one_session() # --max_block_size=1 to make it killable (check the state each 1 second, 1 row) # (the test takes ~40 seconds in debug build, so 60 seconds is ok) query_id=$$-$RANDOM-$SECONDS -${CLICKHOUSE_CLIENT} --max_block_size=1 --format Null --query_id $query_id -q 'SELECT sleepEachRow(1) FROM numbers(600)' & +${CLICKHOUSE_CLIENT} --user=test_01541 --max_block_size=1 --format Null --query_id $query_id -q 'SELECT sleepEachRow(1) FROM numbers(600)' & # trap sleep_query_pid=$! function cleanup() @@ -59,4 +63,6 @@ echo 'TCP' execute_tcp echo 'OK' +${CLICKHOUSE_CLIENT} --query "DROP USER test_01541"; + exit 0 From d99940b7f48ccdf0a1304b8493b4c50f8b22bf2e Mon Sep 17 00:00:00 2001 From: Grigory Buteyko Date: Wed, 4 Nov 2020 12:57:49 +0300 Subject: [PATCH 180/314] Development instructions updated to GCC 10 from GCC 9 --- docs/en/development/build.md | 14 +++++++------- docs/en/development/developer-instruction.md | 8 ++++---- docs/es/development/build.md | 12 ++++++------ docs/es/development/developer-instruction.md | 8 ++++---- docs/fa/development/build.md | 10 +++++----- docs/fa/development/developer-instruction.md | 8 ++++---- docs/fr/development/build.md | 12 ++++++------ docs/fr/development/developer-instruction.md | 8 ++++---- docs/ja/development/build.md | 10 +++++----- docs/ja/development/developer-instruction.md | 4 ++-- docs/ru/development/developer-instruction.md | 4 ++-- docs/tr/development/build.md | 12 ++++++------ docs/tr/development/developer-instruction.md | 4 ++-- docs/zh/development/build.md | 10 +++++----- docs/zh/development/developer-instruction.md | 4 ++-- 15 files changed, 64 insertions(+), 64 deletions(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index e9b6ecebae8..d6e06be3e38 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -23,7 +23,7 @@ $ sudo apt-get install git cmake python ninja-build Or cmake3 instead of cmake on older systems. -### Install GCC 9 {#install-gcc-9} +### Install GCC 10 {#install-gcc-10} There are several ways to do this. @@ -32,7 +32,7 @@ There are several ways to do this. On Ubuntu 19.10 or newer: $ sudo apt-get update - $ sudo apt-get install gcc-9 g++-9 + $ sudo apt-get install gcc-10 g++-10 #### Install from a PPA Package {#install-from-a-ppa-package} @@ -42,18 +42,18 @@ On older Ubuntu: $ sudo apt-get install software-properties-common $ sudo apt-add-repository ppa:ubuntu-toolchain-r/test $ sudo apt-get update -$ sudo apt-get install gcc-9 g++-9 +$ sudo apt-get install gcc-10 g++-10 ``` #### Install from Sources {#install-from-sources} See [utils/ci/build-gcc-from-sources.sh](https://github.com/ClickHouse/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh) -### Use GCC 9 for Builds {#use-gcc-9-for-builds} +### Use GCC 10 for Builds {#use-gcc-10-for-builds} ``` bash -$ export CC=gcc-9 -$ export CXX=g++-9 +$ export CC=gcc-10 +$ export CXX=g++-10 ``` ### Checkout ClickHouse Sources {#checkout-clickhouse-sources} @@ -88,7 +88,7 @@ The build requires the following components: - Git (is used only to checkout the sources, it’s not needed for the build) - CMake 3.10 or newer - Ninja (recommended) or Make -- C++ compiler: gcc 9 or clang 8 or newer +- C++ compiler: gcc 10 or clang 8 or newer - Linker: lld or gold (the classic GNU ld won’t work) - Python (is only used inside LLVM build and it is optional) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index 3dc7c138520..dc95c3ec50b 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -131,13 +131,13 @@ ClickHouse uses several external libraries for building. All of them do not need ## C++ Compiler {#c-compiler} -Compilers GCC starting from version 9 and Clang version 8 or above are supported for building ClickHouse. +Compilers GCC starting from version 10 and Clang version 8 or above are supported for building ClickHouse. Official Yandex builds currently use GCC because it generates machine code of slightly better performance (yielding a difference of up to several percent according to our benchmarks). And Clang is more convenient for development usually. Though, our continuous integration (CI) platform runs checks for about a dozen of build combinations. To install GCC on Ubuntu run: `sudo apt install gcc g++` -Check the version of gcc: `gcc --version`. If it is below 9, then follow the instruction here: https://clickhouse.tech/docs/en/development/build/#install-gcc-9. +Check the version of gcc: `gcc --version`. If it is below 10, then follow the instruction here: https://clickhouse.tech/docs/en/development/build/#install-gcc-10. Mac OS X build is supported only for Clang. Just run `brew install llvm` @@ -152,11 +152,11 @@ Now that you are ready to build ClickHouse we recommend you to create a separate You can have several different directories (build_release, build_debug, etc.) for different types of build. -While inside the `build` directory, configure your build by running CMake. Before the first run, you need to define environment variables that specify compiler (version 9 gcc compiler in this example). +While inside the `build` directory, configure your build by running CMake. Before the first run, you need to define environment variables that specify compiler (version 10 gcc compiler in this example). Linux: - export CC=gcc-9 CXX=g++-9 + export CC=gcc-10 CXX=g++-10 cmake .. Mac OS X: diff --git a/docs/es/development/build.md b/docs/es/development/build.md index 7a1479874eb..42cd9b5433f 100644 --- a/docs/es/development/build.md +++ b/docs/es/development/build.md @@ -19,7 +19,7 @@ $ sudo apt-get install git cmake python ninja-build O cmake3 en lugar de cmake en sistemas más antiguos. -## Instalar GCC 9 {#install-gcc-9} +## Instalar GCC 10 {#install-gcc-10} Hay varias formas de hacer esto. @@ -29,18 +29,18 @@ Hay varias formas de hacer esto. $ sudo apt-get install software-properties-common $ sudo apt-add-repository ppa:ubuntu-toolchain-r/test $ sudo apt-get update -$ sudo apt-get install gcc-9 g++-9 +$ sudo apt-get install gcc-10 g++-10 ``` ### Instalar desde fuentes {#install-from-sources} Mira [Sistema abierto.](https://github.com/ClickHouse/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh) -## Usar GCC 9 para compilaciones {#use-gcc-9-for-builds} +## Usar GCC 10 para compilaciones {#use-gcc-10-for-builds} ``` bash -$ export CC=gcc-9 -$ export CXX=g++-9 +$ export CC=gcc-10 +$ export CXX=g++-10 ``` ## Fuentes de ClickHouse de pago {#checkout-clickhouse-sources} @@ -76,7 +76,7 @@ La compilación requiere los siguientes componentes: - Git (se usa solo para verificar las fuentes, no es necesario para la compilación) - CMake 3.10 o más reciente - Ninja (recomendado) o Hacer -- Compilador de C ++: gcc 9 o clang 8 o más reciente +- Compilador de C ++: gcc 10 o clang 8 o más reciente - Enlazador: lld u oro (el clásico GNU ld no funcionará) - Python (solo se usa dentro de la compilación LLVM y es opcional) diff --git a/docs/es/development/developer-instruction.md b/docs/es/development/developer-instruction.md index 8570fc1924c..390ac55602d 100644 --- a/docs/es/development/developer-instruction.md +++ b/docs/es/development/developer-instruction.md @@ -135,13 +135,13 @@ ClickHouse utiliza varias bibliotecas externas para la construcción. Todos ello # Compilador de C ++ {#c-compiler} -Los compiladores GCC a partir de la versión 9 y Clang versión 8 o superior son compatibles para construir ClickHouse. +Los compiladores GCC a partir de la versión 10 y Clang versión 8 o superior son compatibles para construir ClickHouse. Las compilaciones oficiales de Yandex actualmente usan GCC porque genera código de máquina de un rendimiento ligeramente mejor (con una diferencia de hasta varios por ciento según nuestros puntos de referencia). Y Clang es más conveniente para el desarrollo generalmente. Sin embargo, nuestra plataforma de integración continua (CI) ejecuta verificaciones de aproximadamente una docena de combinaciones de compilación. Para instalar GCC en Ubuntu, ejecute: `sudo apt install gcc g++` -Compruebe la versión de gcc: `gcc --version`. Si está por debajo de 9, siga las instrucciones aquí: https://clickhouse.tech/docs/es/development/build/#install-gcc-9. +Compruebe la versión de gcc: `gcc --version`. Si está por debajo de 9, siga las instrucciones aquí: https://clickhouse.tech/docs/es/development/build/#install-gcc-10. La compilación de Mac OS X solo es compatible con Clang. Sólo tiene que ejecutar `brew install llvm` @@ -156,11 +156,11 @@ Ahora que está listo para construir ClickHouse, le recomendamos que cree un dir Puede tener varios directorios diferentes (build_release, build_debug, etc.) para diferentes tipos de construcción. -Mientras que dentro de la `build` directorio, configure su compilación ejecutando CMake. Antes de la primera ejecución, debe definir variables de entorno que especifiquen el compilador (compilador gcc versión 9 en este ejemplo). +Mientras que dentro de la `build` directorio, configure su compilación ejecutando CMake. Antes de la primera ejecución, debe definir variables de entorno que especifiquen el compilador (compilador gcc versión 10 en este ejemplo). Linux: - export CC=gcc-9 CXX=g++-9 + export CC=gcc-10 CXX=g++-10 cmake .. Mac OS X: diff --git a/docs/fa/development/build.md b/docs/fa/development/build.md index db3aa96285f..b3d4fa1764e 100644 --- a/docs/fa/development/build.md +++ b/docs/fa/development/build.md @@ -20,7 +20,7 @@ $ sudo apt-get install git cmake python ninja-build یا سیمک 3 به جای کیک در سیستم های قدیمی تر. -## نصب شورای همکاری خلیج فارس 9 {#install-gcc-9} +## نصب شورای همکاری خلیج فارس 10 {install-gcc-10} راه های مختلفی برای انجام این کار وجود دارد. @@ -30,18 +30,18 @@ $ sudo apt-get install git cmake python ninja-build $ sudo apt-get install software-properties-common $ sudo apt-add-repository ppa:ubuntu-toolchain-r/test $ sudo apt-get update -$ sudo apt-get install gcc-9 g++-9 +$ sudo apt-get install gcc-10 g++-10 ``` ### نصب از منابع {#install-from-sources} نگاه کن [utils/ci/build-gcc-from-sources.sh](https://github.com/ClickHouse/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh) -## استفاده از شورای همکاری خلیج فارس 9 برای ساخت {#use-gcc-9-for-builds} +## استفاده از شورای همکاری خلیج فارس 10 برای ساخت {#use-gcc-10-for-builds} ``` bash -$ export CC=gcc-9 -$ export CXX=g++-9 +$ export CC=gcc-10 +$ export CXX=g++-10 ``` ## پرداخت منابع کلیک {#checkout-clickhouse-sources} diff --git a/docs/fa/development/developer-instruction.md b/docs/fa/development/developer-instruction.md index c9461748098..01fb24ac006 100644 --- a/docs/fa/development/developer-instruction.md +++ b/docs/fa/development/developer-instruction.md @@ -137,13 +137,13 @@ toc_title: "\u062F\u0633\u062A\u0648\u0631\u0627\u0644\u0639\u0645\u0644 \u062A\ # ج ++ کامپایلر {#c-compiler} -کامپایلر شورای همکاری خلیج فارس با شروع از نسخه 9 و صدای شیپور نسخه 8 یا بالاتر برای ساخت و ساز خانه عروسکی پشتیبانی می کند. +کامپایلر شورای همکاری خلیج فارس با شروع از نسخه 10 و صدای شیپور نسخه 8 یا بالاتر برای ساخت و ساز خانه عروسکی پشتیبانی می کند. یاندکس رسمی ایجاد شده در حال حاضر با استفاده از شورای همکاری خلیج فارس به دلیل تولید کد ماشین از عملکرد کمی بهتر (بازده تفاوت تا چند درصد با توجه به معیار ما). و کلانگ معمولا برای توسعه راحت تر است. هر چند, ادغام مداوم ما (سی) پلت فرم اجرا می شود چک برای حدود یک دوجین از ترکیب ساخت. برای نصب شورای همکاری خلیج فارس در اوبونتو اجرای: `sudo apt install gcc g++` -بررسی نسخه شورای همکاری خلیج فارس: `gcc --version`. اگر زیر است 9, سپس دستورالعمل اینجا را دنبال کنید: https://clickhouse.tech/docs/fa/development/build/#install-gcc-9. +بررسی نسخه شورای همکاری خلیج فارس: `gcc --version`. اگر زیر است 9, سپس دستورالعمل اینجا را دنبال کنید: https://clickhouse.tech/docs/fa/development/build/#install-gcc-10. سیستم عامل مک ایکس ساخت فقط برای صدای جرنگ جرنگ پشتیبانی می شود. فقط فرار کن `brew install llvm` @@ -158,11 +158,11 @@ toc_title: "\u062F\u0633\u062A\u0648\u0631\u0627\u0644\u0639\u0645\u0644 \u062A\ شما می توانید چندین دایرکتوری های مختلف (build_release, build_debug ، ) برای انواع مختلف ساخت. -در حالی که در داخل `build` فهرست, پیکربندی ساخت خود را با در حال اجرا کیک. قبل از اولین اجرا, شما نیاز به تعریف متغیرهای محیطی که کامپایلر را مشخص (نسخه 9 کامپایلر شورای همکاری خلیج فارس در این مثال). +در حالی که در داخل `build` فهرست, پیکربندی ساخت خود را با در حال اجرا کیک. قبل از اولین اجرا, شما نیاز به تعریف متغیرهای محیطی که کامپایلر را مشخص (نسخه 10 کامپایلر شورای همکاری خلیج فارس در این مثال). لینوکس: - export CC=gcc-9 CXX=g++-9 + export CC=gcc-10 CXX=g++-10 cmake .. سیستم عامل مک ایکس: diff --git a/docs/fr/development/build.md b/docs/fr/development/build.md index 0d2713aaaa5..4889373c52a 100644 --- a/docs/fr/development/build.md +++ b/docs/fr/development/build.md @@ -19,7 +19,7 @@ $ sudo apt-get install git cmake python ninja-build Ou cmake3 au lieu de cmake sur les systèmes plus anciens. -## Installer GCC 9 {#install-gcc-9} +## Installer GCC 10 {#install-gcc-10} Il y a plusieurs façons de le faire. @@ -29,18 +29,18 @@ Il y a plusieurs façons de le faire. $ sudo apt-get install software-properties-common $ sudo apt-add-repository ppa:ubuntu-toolchain-r/test $ sudo apt-get update -$ sudo apt-get install gcc-9 g++-9 +$ sudo apt-get install gcc-10 g++-10 ``` ### Installer à partir de Sources {#install-from-sources} Regarder [utils/ci/build-gcc-from-sources.sh](https://github.com/ClickHouse/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh) -## Utilisez GCC 9 pour les Builds {#use-gcc-9-for-builds} +## Utilisez GCC 10 pour les Builds {#use-gcc-10-for-builds} ``` bash -$ export CC=gcc-9 -$ export CXX=g++-9 +$ export CC=gcc-10 +$ export CXX=g++-10 ``` ## Commander Clickhouse Sources {#checkout-clickhouse-sources} @@ -76,7 +76,7 @@ La construction nécessite les composants suivants: - Git (est utilisé uniquement pour extraire les sources, ce n'est pas nécessaire pour la construction) - CMake 3.10 ou plus récent - Ninja (recommandé) ou faire -- Compilateur C++: gcc 9 ou clang 8 ou plus récent +- Compilateur C++: gcc 10 ou clang 8 ou plus récent - Linker: lld ou gold (le classique GNU LD ne fonctionnera pas) - Python (est seulement utilisé dans la construction LLVM et il est facultatif) diff --git a/docs/fr/development/developer-instruction.md b/docs/fr/development/developer-instruction.md index 576188941bd..e78ed7ba6d9 100644 --- a/docs/fr/development/developer-instruction.md +++ b/docs/fr/development/developer-instruction.md @@ -135,13 +135,13 @@ ClickHouse utilise plusieurs bibliothèques externes pour la construction. Tous # Compilateur C++ {#c-compiler} -Les compilateurs GCC à partir de la version 9 et Clang version 8 ou supérieure sont pris en charge pour construire ClickHouse. +Les compilateurs GCC à partir de la version 10 et Clang version 8 ou supérieure sont pris en charge pour construire ClickHouse. Les builds officiels de Yandex utilisent actuellement GCC car ils génèrent du code machine de performances légèrement meilleures (ce qui donne une différence allant jusqu'à plusieurs pour cent selon nos benchmarks). Et Clang est plus pratique pour le développement habituellement. Cependant, notre plate-forme d'intégration continue (CI) vérifie environ une douzaine de combinaisons de construction. Pour installer GCC sur Ubuntu Exécutez: `sudo apt install gcc g++` -Vérifiez la version de gcc: `gcc --version`. Si elle est inférieure à 9, suivez les instructions ici: https://clickhouse.tech/docs/fr/development/build/#install-gcc-9. +Vérifiez la version de gcc: `gcc --version`. Si elle est inférieure à 10, suivez les instructions ici: https://clickhouse.tech/docs/fr/development/build/#install-gcc-10. Mac OS X build est pris en charge uniquement pour Clang. Il suffit d'exécuter `brew install llvm` @@ -156,11 +156,11 @@ Maintenant que vous êtes prêt à construire ClickHouse nous vous conseillons d Vous pouvez avoir plusieurs répertoires différents (build_release, build_debug, etc.) pour les différents types de construction. -Tandis qu'à l'intérieur de la `build` répertoire, configurez votre build en exécutant CMake. Avant la première exécution, vous devez définir des variables d'environnement qui spécifient le compilateur (compilateur gcc version 9 dans cet exemple). +Tandis qu'à l'intérieur de la `build` répertoire, configurez votre build en exécutant CMake. Avant la première exécution, vous devez définir des variables d'environnement qui spécifient le compilateur (compilateur gcc version 10 dans cet exemple). Linux: - export CC=gcc-9 CXX=g++-9 + export CC=gcc-10 CXX=g++-10 cmake .. Mac OS X: diff --git a/docs/ja/development/build.md b/docs/ja/development/build.md index 33709cb65f3..e44ba45485e 100644 --- a/docs/ja/development/build.md +++ b/docs/ja/development/build.md @@ -19,7 +19,7 @@ $ sudo apt-get install git cmake python ninja-build 古いシステムではcmakeの代わりにcmake3。 -## GCC9のインストール {#install-gcc-9} +## GCC9のインストール {#install-gcc-10} これを行うにはいくつかの方法があります。 @@ -29,18 +29,18 @@ $ sudo apt-get install git cmake python ninja-build $ sudo apt-get install software-properties-common $ sudo apt-add-repository ppa:ubuntu-toolchain-r/test $ sudo apt-get update -$ sudo apt-get install gcc-9 g++-9 +$ sudo apt-get install gcc-10 g++-10 ``` ### ソースからインスト {#install-from-sources} 見て [utils/ci/build-gcc-from-sources.sh](https://github.com/ClickHouse/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh) -## ビルドにGCC9を使用する {#use-gcc-9-for-builds} +## ビルドにGCC9を使用する {#use-gcc-10-for-builds} ``` bash -$ export CC=gcc-9 -$ export CXX=g++-9 +$ export CC=gcc-10 +$ export CXX=g++-10 ``` ## ツつィツ姪"ツ債ツつケ {#checkout-clickhouse-sources} diff --git a/docs/ja/development/developer-instruction.md b/docs/ja/development/developer-instruction.md index f3a703f9879..988becf98c3 100644 --- a/docs/ja/development/developer-instruction.md +++ b/docs/ja/development/developer-instruction.md @@ -141,7 +141,7 @@ ClickHouseのビルドには、バージョン9以降のGCCとClangバージョ UBUNTUにGCCをインストールするには: `sudo apt install gcc g++` -Gccのバージョンを確認する: `gcc --version`. の場合は下記9その指示に従う。https://clickhouse.tech/docs/ja/development/build/#install-gcc-9. +Gccのバージョンを確認する: `gcc --version`. の場合は下記9その指示に従う。https://clickhouse.tech/docs/ja/development/build/#install-gcc-10. Mac OS XのビルドはClangでのみサポートされています。 ちょうど実行 `brew install llvm` @@ -160,7 +160,7 @@ ClickHouseを構築する準備ができたので、別のディレクトリを Linux: - export CC=gcc-9 CXX=g++-9 + export CC=gcc-10 CXX=g++-10 cmake .. Mac OS X: diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index b0c84d9de7c..4bdcf89004d 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -142,7 +142,7 @@ ClickHouse использует для сборки некоторое коли Для установки GCC под Ubuntu, выполните: `sudo apt install gcc g++`. -Проверьте версию gcc: `gcc --version`. Если версия меньше 9, то следуйте инструкции: https://clickhouse.tech/docs/ru/development/build/#install-gcc-9. +Проверьте версию gcc: `gcc --version`. Если версия меньше 10, то следуйте инструкции: https://clickhouse.tech/docs/ru/development/build/#install-gcc-10. Сборка под Mac OS X поддерживается только для компилятора Clang. Чтобы установить его выполните `brew install llvm` @@ -162,7 +162,7 @@ ClickHouse использует для сборки некоторое коли Linux: - export CC=gcc-9 CXX=g++-9 + export CC=gcc-10 CXX=g++-10 cmake .. Mac OS X: diff --git a/docs/tr/development/build.md b/docs/tr/development/build.md index 92eaaf30ced..93ddebffa75 100644 --- a/docs/tr/development/build.md +++ b/docs/tr/development/build.md @@ -19,7 +19,7 @@ $ sudo apt-get install git cmake python ninja-build Veya eski sistemlerde cmake yerine cmake3. -## Gcc 9'u yükle {#install-gcc-9} +## Gcc 10'u yükle {#install-gcc-10} Bunu yapmak için çeşitli yollar vardır. @@ -29,18 +29,18 @@ Bunu yapmak için çeşitli yollar vardır. $ sudo apt-get install software-properties-common $ sudo apt-add-repository ppa:ubuntu-toolchain-r/test $ sudo apt-get update -$ sudo apt-get install gcc-9 g++-9 +$ sudo apt-get install gcc-10 g++-10 ``` ### Kaynaklardan yükleyin {#install-from-sources} Bakmak [utils/ci/build-gcc-from-sources.sh](https://github.com/ClickHouse/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh) -## Yapılar için GCC 9 kullanın {#use-gcc-9-for-builds} +## Yapılar için GCC 10 kullanın {#use-gcc-10-for-builds} ``` bash -$ export CC=gcc-9 -$ export CXX=g++-9 +$ export CC=gcc-10 +$ export CXX=g++-10 ``` ## Checkout ClickHouse Kaynakları {#checkout-clickhouse-sources} @@ -76,7 +76,7 @@ Yapı aşağıdaki bileşenleri gerektirir: - Git (yalnızca kaynakları kontrol etmek için kullanılır, yapı için gerekli değildir) - Cmake 3.10 veya daha yeni - Ninja (önerilir) veya yapmak -- C ++ derleyici: gcc 9 veya clang 8 veya daha yeni +- C ++ derleyici: gcc 10 veya clang 8 veya daha yeni - Linker :lld veya altın (klasik GNU ld çalışmaz) - Python (sadece LLVM yapısında kullanılır ve isteğe bağlıdır) diff --git a/docs/tr/development/developer-instruction.md b/docs/tr/development/developer-instruction.md index 10b1ea35758..51a6c4345c6 100644 --- a/docs/tr/development/developer-instruction.md +++ b/docs/tr/development/developer-instruction.md @@ -141,7 +141,7 @@ Resmi Yandex şu anda GCC'Yİ kullanıyor çünkü biraz daha iyi performansa sa Ubuntu run GCC yüklemek için: `sudo apt install gcc g++` -Gcc sürümünü kontrol edin: `gcc --version`. 9'un altındaysa, buradaki talimatları izleyin: https://clickhouse.tech/docs/tr/development/build/#install-gcc-9. +Gcc sürümünü kontrol edin: `gcc --version`. 10'un altındaysa, buradaki talimatları izleyin: https://clickhouse.tech/docs/tr/development/build/#install-gcc-10. Mac OS X build sadece Clang için desteklenir. Sadece koş `brew install llvm` @@ -160,7 +160,7 @@ Birkaç farklı dizine (build_release, build_debug, vb.) sahip olabilirsiniz.) f Linux: - export CC=gcc-9 CXX=g++-9 + export CC=gcc-10 CXX=g++-10 cmake .. Mac OS X: diff --git a/docs/zh/development/build.md b/docs/zh/development/build.md index 8cf371b7be5..1aa5c1c97b7 100644 --- a/docs/zh/development/build.md +++ b/docs/zh/development/build.md @@ -35,7 +35,7 @@ sudo apt-get install git cmake ninja-build 或cmake3而不是旧系统上的cmake。 或者在早期版本的系统中用 cmake3 替代 cmake -## 安装 GCC 9 {#an-zhuang-gcc-9} +## 安装 GCC 10 {#an-zhuang-gcc-10} 有几种方法可以做到这一点。 @@ -45,18 +45,18 @@ sudo apt-get install git cmake ninja-build sudo apt-get install software-properties-common sudo apt-add-repository ppa:ubuntu-toolchain-r/test sudo apt-get update -sudo apt-get install gcc-9 g++-9 +sudo apt-get install gcc-10 g++-10 ``` ### 源码安装 gcc {#yuan-ma-an-zhuang-gcc} 请查看 [utils/ci/build-gcc-from-sources.sh](https://github.com/ClickHouse/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh) -## 使用 GCC 9 来编译 {#shi-yong-gcc-9-lai-bian-yi} +## 使用 GCC 10 来编译 {#shi-yong-gcc-10-lai-bian-yi} ``` bash -export CC=gcc-9 -export CXX=g++-9 +export CC=gcc-10 +export CXX=g++-10 ``` ## 拉取 ClickHouse 源码 {#la-qu-clickhouse-yuan-ma-1} diff --git a/docs/zh/development/developer-instruction.md b/docs/zh/development/developer-instruction.md index faa333f8e31..3e2ccf5da35 100644 --- a/docs/zh/development/developer-instruction.md +++ b/docs/zh/development/developer-instruction.md @@ -129,7 +129,7 @@ Yandex官方当前使用GCC构建ClickHouse,因为它生成的机器代码性 在Ubuntu上安装GCC,请执行:`sudo apt install gcc g++` -请使用`gcc --version`查看gcc的版本。如果gcc版本低于9,请参考此处的指示:https://clickhouse.tech/docs/zh/development/build/#an-zhuang-gcc-9 。 +请使用`gcc --version`查看gcc的版本。如果gcc版本低于9,请参考此处的指示:https://clickhouse.tech/docs/zh/development/build/#an-zhuang-gcc-10 。 在Mac OS X上安装GCC,请执行:`brew install gcc` @@ -146,7 +146,7 @@ Yandex官方当前使用GCC构建ClickHouse,因为它生成的机器代码性 在`build`目录下,通过运行CMake配置构建。 在第一次运行之前,请定义用于指定编译器的环境变量(本示例中为gcc 9 编译器)。 - export CC=gcc-9 CXX=g++-9 + export CC=gcc-10 CXX=g++-10 cmake .. `CC`变量指代C的编译器(C Compiler的缩写),而`CXX`变量指代要使用哪个C++编译器进行编译。 From f7c77b4a25772dde0a88008e26b4e80b23bbfad7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 13:14:23 +0300 Subject: [PATCH 181/314] Fix unit tests --- src/Common/tests/gtest_global_register.h | 15 +++++++++++ src/Functions/tests/gtest_abtesting.cpp | 27 +++++++++++-------- .../MySQL/tests/gtest_create_rewritten.cpp | 4 ++- src/Storages/tests/gtest_storage_log.cpp | 3 +++ ..._transform_query_for_external_database.cpp | 2 +- 5 files changed, 38 insertions(+), 13 deletions(-) create mode 100644 src/Common/tests/gtest_global_register.h diff --git a/src/Common/tests/gtest_global_register.h b/src/Common/tests/gtest_global_register.h new file mode 100644 index 00000000000..c4bde825109 --- /dev/null +++ b/src/Common/tests/gtest_global_register.h @@ -0,0 +1,15 @@ +#pragma once + +#include +#include + + +inline void tryRegisterFunctions() +{ + static struct Register { Register() { DB::registerFunctions(); } } registered; +} + +inline void tryRegisterFormats() +{ + static struct Register { Register() { DB::registerFormats(); } } registered; +} diff --git a/src/Functions/tests/gtest_abtesting.cpp b/src/Functions/tests/gtest_abtesting.cpp index b388a187479..e7ef5b5c3cf 100644 --- a/src/Functions/tests/gtest_abtesting.cpp +++ b/src/Functions/tests/gtest_abtesting.cpp @@ -10,39 +10,44 @@ Variants test_bayesab(std::string dist, PODArray xs, PODArray { Variants variants; - std::cout << std::fixed; + //std::cout << std::fixed; if (dist == "beta") { - std::cout << dist << "\nclicks: "; - for (auto x : xs) std::cout << x << " "; +/* std::cout << dist << "\nclicks: "; + for (auto x : xs) + std::cout << x << " "; std::cout <<"\tconversions: "; - for (auto y : ys) std::cout << y << " "; + for (auto y : ys) + std::cout << y << " "; - std::cout << "\n"; + std::cout << "\n";*/ variants = bayesian_ab_test(dist, xs, ys); } else if (dist == "gamma") { - std::cout << dist << "\nclicks: "; - for (auto x : xs) std::cout << x << " "; +/* std::cout << dist << "\nclicks: "; + for (auto x : xs) + std::cout << x << " "; std::cout <<"\tcost: "; - for (auto y : ys) std::cout << y << " "; + for (auto y : ys) + std::cout << y << " "; + + std::cout << "\n";*/ - std::cout << "\n"; variants = bayesian_ab_test(dist, xs, ys); } - for (size_t i = 0; i < variants.size(); ++i) +/* for (size_t i = 0; i < variants.size(); ++i) std::cout << i << " beats 0: " << variants[i].beats_control << std::endl; for (size_t i = 0; i < variants.size(); ++i) std::cout << i << " to be best: " << variants[i].best << std::endl; std::cout << convertToJson({"0", "1", "2"}, variants) << std::endl; - +*/ Float64 max_val = 0.0, min_val = 2.0; for (size_t i = 0; i < variants.size(); ++i) { diff --git a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp index b940e4e0c95..2221b7b1588 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp @@ -12,7 +12,9 @@ #include #include #include -#include +#include +#include + using namespace DB; diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index 8de14b53471..7dc140c2da8 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include @@ -127,6 +128,8 @@ std::string readData(DB::StoragePtr & table, const DB::Context & context) sample.insert(std::move(col)); } + tryRegisterFormats(); + std::ostringstream ss; WriteBufferFromOStream out_buf(ss); BlockOutputStreamPtr output = FormatFactory::instance().getOutput("Values", out_buf, sample, context); diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 31fc49582ad..48811c1c86a 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include using namespace DB; From 4dccc51332bb6361a7fe6d7df5a97ddb66f3bf9f Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 4 Nov 2020 18:21:59 +0800 Subject: [PATCH 182/314] Fix typo --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 29052f7ba84..4d4712dcba7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -109,7 +109,7 @@ class IColumn; \ M(UInt64, parallel_distributed_insert_select, 0, "Process distributed INSERT SELECT query in the same cluster on local tables on every shard, if 1 SELECT is executed on each shard, if 2 SELECT and INSERT is executed on each shard", 0) \ M(UInt64, distributed_group_by_no_merge, 0, "If 1, Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards. If 2 - same as 1 but also apply ORDER BY and LIMIT stages", 0) \ - M(Bool, optimize_distributed_group_by_sharding_key, false, "Optimize GROUP BY sharding_key queries (by avodiing costly aggregation on the initiator server).", 0) \ + M(Bool, optimize_distributed_group_by_sharding_key, false, "Optimize GROUP BY sharding_key queries (by avoiding costly aggregation on the initiator server).", 0) \ M(Bool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \ M(Bool, allow_nondeterministic_optimize_skip_unused_shards, false, "Allow non-deterministic functions (includes dictGet) in sharding_key for optimize_skip_unused_shards", 0) \ M(UInt64, force_optimize_skip_unused_shards, 0, "Throw an exception if unused shards cannot be skipped (1 - throw only if the table has the sharding key, 2 - always throw.", 0) \ From ff8f80b6b6d446a6daff9d1962da081197964fc9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 13:34:47 +0300 Subject: [PATCH 183/314] Remove old file --- .../tests/gtest_global_register_functions.h | 18 ------------------ 1 file changed, 18 deletions(-) delete mode 100644 src/Common/tests/gtest_global_register_functions.h diff --git a/src/Common/tests/gtest_global_register_functions.h b/src/Common/tests/gtest_global_register_functions.h deleted file mode 100644 index 5ca4d64522e..00000000000 --- a/src/Common/tests/gtest_global_register_functions.h +++ /dev/null @@ -1,18 +0,0 @@ -#pragma once -#include -#include - -struct RegisteredFunctionsState -{ - RegisteredFunctionsState() - { - DB::registerFunctions(); - } - - RegisteredFunctionsState(RegisteredFunctionsState &&) = default; -}; - -inline void tryRegisterFunctions() -{ - static RegisteredFunctionsState registered_functions_state; -} From 85f81f9d9e64d7456f264f95a84371ca3d04d2da Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 13:38:04 +0300 Subject: [PATCH 184/314] Minor modification: less templates --- .../AggregateFunctionOrFill.cpp | 29 ++++++++++++------- 1 file changed, 18 insertions(+), 11 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionOrFill.cpp b/src/AggregateFunctions/AggregateFunctionOrFill.cpp index af107e26ca9..959836c0e49 100644 --- a/src/AggregateFunctions/AggregateFunctionOrFill.cpp +++ b/src/AggregateFunctions/AggregateFunctionOrFill.cpp @@ -9,16 +9,23 @@ namespace DB namespace { -template +enum class Kind +{ + OrNull, + OrDefault +}; + class AggregateFunctionCombinatorOrFill final : public IAggregateFunctionCombinator { +private: + Kind kind; + public: + AggregateFunctionCombinatorOrFill(Kind kind_) : kind(kind_) {} + String getName() const override { - if constexpr (UseNull) - return "OrNull"; - else - return "OrDefault"; + return kind == Kind::OrNull ? "OrNull" : "OrDefault"; } AggregateFunctionPtr transformAggregateFunction( @@ -27,10 +34,10 @@ public: const DataTypes & arguments, const Array & params) const override { - return std::make_shared>( - nested_function, - arguments, - params); + if (kind == Kind::OrNull) + return std::make_shared>(nested_function, arguments, params); + else + return std::make_shared>(nested_function, arguments, params); } }; @@ -38,8 +45,8 @@ public: void registerAggregateFunctionCombinatorOrFill(AggregateFunctionCombinatorFactory & factory) { - factory.registerCombinator(std::make_shared>()); - factory.registerCombinator(std::make_shared>()); + factory.registerCombinator(std::make_shared(Kind::OrNull)); + factory.registerCombinator(std::make_shared(Kind::OrDefault)); } } From 155c11d29c793d311bb10f278d47ba71c1ef9eb6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 13:53:10 +0300 Subject: [PATCH 185/314] Optimize -OrNull and -OrDefault aggregate functions by providing specializations #16123 --- .../AggregateFunctionOrFill.h | 29 +++++++++++++++++-- 1 file changed, 27 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionOrFill.h b/src/AggregateFunctions/AggregateFunctionOrFill.h index 333f07d5e33..58b2cebb9e4 100644 --- a/src/AggregateFunctions/AggregateFunctionOrFill.h +++ b/src/AggregateFunctions/AggregateFunctionOrFill.h @@ -78,7 +78,6 @@ public: void create(AggregateDataPtr place) const override { nested_function->create(place); - place[size_of_data] = 0; } @@ -94,10 +93,36 @@ public: Arena * arena) const override { nested_function->add(place, columns, row_num, arena); - place[size_of_data] = 1; } + void addBatch(size_t batch_size, AggregateDataPtr * places, size_t place_offset, const IColumn ** columns, Arena * arena) const override + { + nested_function->addBatch(batch_size, places, place_offset, columns, arena); + for (size_t i = 0; i < batch_size; ++i) + (places[i] + place_offset)[size_of_data] = 1; + } + + void addBatchSinglePlace(size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const override + { + nested_function->addBatchSinglePlace(batch_size, place, columns, arena); + place[size_of_data] = 1; + } + + void addBatchSinglePlaceNotNull( + size_t batch_size, AggregateDataPtr place, const IColumn ** columns, const UInt8 * null_map, Arena * arena) const override + { + nested_function->addBatchSinglePlaceNotNull(batch_size, place, columns, null_map, arena); + for (size_t i = 0; i < batch_size; ++i) + { + if (!null_map[i]) + { + place[size_of_data] = 1; + break; + } + } + } + void merge( AggregateDataPtr place, ConstAggregateDataPtr rhs, From 938ef62532776ecc8d6449a064d1d5d4d1446152 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 14:00:43 +0300 Subject: [PATCH 186/314] Add a test --- tests/performance/or_null_default.xml | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 tests/performance/or_null_default.xml diff --git a/tests/performance/or_null_default.xml b/tests/performance/or_null_default.xml new file mode 100644 index 00000000000..009719f66a5 --- /dev/null +++ b/tests/performance/or_null_default.xml @@ -0,0 +1,5 @@ + + SELECT sumOrNull(number) FROM numbers(100000000) + SELECT sumOrDefault(toNullable(number)) FROM numbers(100000000) + SELECT sumOrNull(number) FROM numbers(10000000) GROUP BY number % 1024 + From c7618ea99e84d70362d9f400e81660191a9fe8fe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 14:02:12 +0300 Subject: [PATCH 187/314] Remove obsolete code from JOIN --- src/Interpreters/TableJoin.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index cd837cc15d6..5db914bc457 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -28,8 +28,6 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_) , temporary_files_codec(settings.temporary_files_codec) , tmp_volume(tmp_volume_) { - if (settings.partial_merge_join) - join_algorithm = JoinAlgorithm::PREFER_PARTIAL_MERGE; } void TableJoin::resetCollected() From c7facedf10d720989d22379ce19cc6d7d00ac4e9 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Wed, 4 Nov 2020 12:02:30 +0100 Subject: [PATCH 188/314] Update clickhouse-copier.md --- docs/en/operations/utilities/clickhouse-copier.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/operations/utilities/clickhouse-copier.md b/docs/en/operations/utilities/clickhouse-copier.md index d450f5753e4..ec5a619b86b 100644 --- a/docs/en/operations/utilities/clickhouse-copier.md +++ b/docs/en/operations/utilities/clickhouse-copier.md @@ -7,6 +7,9 @@ toc_title: clickhouse-copier Copies data from the tables in one cluster to tables in another (or the same) cluster. +!!! warning "Warning" + To get a consistent copy, the data in the source tables and partitions should not change during the entire process. + You can run multiple `clickhouse-copier` instances on different servers to perform the same job. ZooKeeper is used for syncing the processes. After starting, `clickhouse-copier`: From c848deed1e123040c1f605cf8eb162cdf7c17e20 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 14:07:54 +0300 Subject: [PATCH 189/314] Fix test --- .../AggregateFunctionOrFill.h | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionOrFill.h b/src/AggregateFunctions/AggregateFunctionOrFill.h index 58b2cebb9e4..456c0e22305 100644 --- a/src/AggregateFunctions/AggregateFunctionOrFill.h +++ b/src/AggregateFunctions/AggregateFunctionOrFill.h @@ -105,20 +105,26 @@ public: void addBatchSinglePlace(size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const override { - nested_function->addBatchSinglePlace(batch_size, place, columns, arena); - place[size_of_data] = 1; + if (batch_size) + { + nested_function->addBatchSinglePlace(batch_size, place, columns, arena); + place[size_of_data] = 1; + } } void addBatchSinglePlaceNotNull( size_t batch_size, AggregateDataPtr place, const IColumn ** columns, const UInt8 * null_map, Arena * arena) const override { - nested_function->addBatchSinglePlaceNotNull(batch_size, place, columns, null_map, arena); - for (size_t i = 0; i < batch_size; ++i) + if (batch_size) { - if (!null_map[i]) + nested_function->addBatchSinglePlaceNotNull(batch_size, place, columns, null_map, arena); + for (size_t i = 0; i < batch_size; ++i) { - place[size_of_data] = 1; - break; + if (!null_map[i]) + { + place[size_of_data] = 1; + break; + } } } } @@ -129,7 +135,6 @@ public: Arena * arena) const override { nested_function->merge(place, rhs, arena); - place[size_of_data] |= rhs[size_of_data]; } From 3993636d8db09c22d52cac76801db6a47e6e38b3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 14:09:20 +0300 Subject: [PATCH 190/314] Add a test --- .../0_stateless/01555_or_fill.reference | 2 ++ tests/queries/0_stateless/01555_or_fill.sql | 22 +++++++++++++++++++ 2 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/01555_or_fill.reference create mode 100644 tests/queries/0_stateless/01555_or_fill.sql diff --git a/tests/queries/0_stateless/01555_or_fill.reference b/tests/queries/0_stateless/01555_or_fill.reference new file mode 100644 index 00000000000..6a204045944 --- /dev/null +++ b/tests/queries/0_stateless/01555_or_fill.reference @@ -0,0 +1,2 @@ +0 \N 0 \N +0 \N 0 \N diff --git a/tests/queries/0_stateless/01555_or_fill.sql b/tests/queries/0_stateless/01555_or_fill.sql new file mode 100644 index 00000000000..a2da07d0a7c --- /dev/null +++ b/tests/queries/0_stateless/01555_or_fill.sql @@ -0,0 +1,22 @@ +SELECT + count(), + countOrNull(), + sum(x), + sumOrNull(x) +FROM +( + SELECT number AS x + FROM numbers(10) + WHERE number > 10 +); + +SELECT + count(), + countOrNull(), + sum(x), + sumOrNull(x) +FROM +( + SELECT 1 AS x + WHERE 0 +); From 7ec73e1f206f34627e48279a7d1eb9da3646e0ef Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 4 Nov 2020 11:21:59 +0000 Subject: [PATCH 191/314] fix build --- src/Interpreters/TableJoin.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index cd837cc15d6..5db914bc457 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -28,8 +28,6 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_) , temporary_files_codec(settings.temporary_files_codec) , tmp_volume(tmp_volume_) { - if (settings.partial_merge_join) - join_algorithm = JoinAlgorithm::PREFER_PARTIAL_MERGE; } void TableJoin::resetCollected() From e0d962a3cfb6404432abda4f579820708e96db11 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 14:22:02 +0300 Subject: [PATCH 192/314] Add a test for #8772 --- tests/performance/group_by_sundy_li.xml | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 tests/performance/group_by_sundy_li.xml diff --git a/tests/performance/group_by_sundy_li.xml b/tests/performance/group_by_sundy_li.xml new file mode 100644 index 00000000000..4c9e397520a --- /dev/null +++ b/tests/performance/group_by_sundy_li.xml @@ -0,0 +1,25 @@ + + +CREATE TABLE a +( + d Date, + os String +) +ENGINE = MergeTree +PARTITION BY d +ORDER BY d + + + insert into a select '2000-01-01', ['aa','bb','cc','dd'][number % 4 + 1] from numbers(100000000) + insert into a select '2000-01-02', ['aa','bb','cc','dd'][number % 4 + 1] from numbers(100000000) + insert into a select '2000-01-03', ['aa','bb','cc','dd'][number % 4 + 1] from numbers(100000000) + insert into a select '2000-01-04', ['aa','bb','cc','dd'][number % 4 + 1] from numbers(100000000) + + OPTIMIZE TABLE a FINAL + + select d, count() from a group by d + select os, count() from a group by os + select d, os, count() from a group by d, os + + drop table if exists a + From 7986dbdfc7a6b57e9924171b46ca9899beb2ad8c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 4 Nov 2020 14:32:55 +0300 Subject: [PATCH 193/314] Update ontime.md --- docs/en/getting-started/example-datasets/ontime.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/ontime.md b/docs/en/getting-started/example-datasets/ontime.md index c2c8d5d930e..6f2408af3b6 100644 --- a/docs/en/getting-started/example-datasets/ontime.md +++ b/docs/en/getting-started/example-datasets/ontime.md @@ -148,7 +148,7 @@ SETTINGS index_granularity = 8192; Loading data: ``` bash -$ for i in *.zip; do echo $i; unzip -cq $i '*.csv' | sed 's/\.00//g' | clickhouse-client --host=example-perftest01j --query="INSERT INTO ontime FORMAT CSVWithNames"; done +$ for i in *.zip; do echo $i; unzip -cq $i '*.csv' | sed 's/\.00//g' | clickhouse-client --input_format_with_names_use_header=0 --host=example-perftest01j --query="INSERT INTO ontime FORMAT CSVWithNames"; done ``` ## Download of Prepared Partitions {#download-of-prepared-partitions} From 269e96f17ee8ffd0f02081d20b004fb6bfbe7cf0 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 4 Nov 2020 12:15:31 +0000 Subject: [PATCH 194/314] fix --- src/Interpreters/TreeRewriter.cpp | 8 +++++--- ...8_setting_aggregate_functions_null_for_empty.reference | 8 ++++++++ .../01528_setting_aggregate_functions_null_for_empty.sql | 8 ++++++++ 3 files changed, 21 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 7c37365a384..c8691c25f1b 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -120,10 +120,12 @@ struct CustomizeAggregateFunctionsSuffixData void visit(ASTFunction & func, ASTPtr &) { - if (AggregateFunctionFactory::instance().isAggregateFunctionName(func.name) - && !endsWith(func.name, customized_func_suffix)) + const auto & instance = AggregateFunctionFactory::instance(); + if (instance.isAggregateFunctionName(func.name) && !endsWith(func.name, customized_func_suffix)) { - func.name = func.name + customized_func_suffix; + auto properties = instance.tryGetProperties(func.name); + if (properties && !properties->returns_default_when_only_null) + func.name = func.name + customized_func_suffix; } } }; diff --git a/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference index 570ea20ffad..9c6ae9c65ab 100644 --- a/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference +++ b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference @@ -1,8 +1,16 @@ 0 \N +0 +\N \N \N +0 +\N 45 45 +10 +10 45 45 +10 +10 diff --git a/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql index c59b592e701..e76ce667bbc 100644 --- a/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql +++ b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql @@ -7,11 +7,15 @@ CREATE TABLE defaults SELECT sum(n) FROM defaults; SELECT sumOrNull(n) FROM defaults; +SELECT count(n) FROM defaults; +SELECT countOrNull(n) FROM defaults; SET aggregate_functions_null_for_empty=1; SELECT sum(n) FROM defaults; SELECT sumOrNull(n) FROM defaults; +SELECT count(n) FROM defaults; +SELECT countOrNull(n) FROM defaults; INSERT INTO defaults SELECT * FROM numbers(10); @@ -19,10 +23,14 @@ SET aggregate_functions_null_for_empty=0; SELECT sum(n) FROM defaults; SELECT sumOrNull(n) FROM defaults; +SELECT count(n) FROM defaults; +SELECT countOrNull(n) FROM defaults; SET aggregate_functions_null_for_empty=1; SELECT sum(n) FROM defaults; SELECT sumOrNull(n) FROM defaults; +SELECT count(n) FROM defaults; +SELECT countOrNull(n) FROM defaults; DROP TABLE defaults; From eedf4439bbefe4ebdaa4233f25ff702090039ee9 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Wed, 4 Nov 2020 10:16:49 -0400 Subject: [PATCH 195/314] more tests for CTE #16575 --- ...5_subqueries_in_with_statement_2.reference | 26 ++++++ .../01495_subqueries_in_with_statement_2.sql | 49 +++++++++++ ...5_subqueries_in_with_statement_3.reference | 8 ++ .../01495_subqueries_in_with_statement_3.sql | 82 +++++++++++++++++++ 4 files changed, 165 insertions(+) create mode 100644 tests/queries/0_stateless/01495_subqueries_in_with_statement_2.reference create mode 100644 tests/queries/0_stateless/01495_subqueries_in_with_statement_2.sql create mode 100644 tests/queries/0_stateless/01495_subqueries_in_with_statement_3.reference create mode 100644 tests/queries/0_stateless/01495_subqueries_in_with_statement_3.sql diff --git a/tests/queries/0_stateless/01495_subqueries_in_with_statement_2.reference b/tests/queries/0_stateless/01495_subqueries_in_with_statement_2.reference new file mode 100644 index 00000000000..d8eaf6f67b6 --- /dev/null +++ b/tests/queries/0_stateless/01495_subqueries_in_with_statement_2.reference @@ -0,0 +1,26 @@ +0 +1 +2 +3 +4 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +1 +1 +3 +3 +1 +3 diff --git a/tests/queries/0_stateless/01495_subqueries_in_with_statement_2.sql b/tests/queries/0_stateless/01495_subqueries_in_with_statement_2.sql new file mode 100644 index 00000000000..7ec4eeaa778 --- /dev/null +++ b/tests/queries/0_stateless/01495_subqueries_in_with_statement_2.sql @@ -0,0 +1,49 @@ + +WITH +x AS (SELECT number AS a FROM numbers(10)), +y AS (SELECT number AS a FROM numbers(5)) +SELECT * FROM x WHERE a in (SELECT a FROM y) +ORDER BY a; + +WITH +x AS (SELECT number AS a FROM numbers(10)), +y AS (SELECT number AS a FROM numbers(5)) +SELECT * FROM x left JOIN y USING a +ORDER BY a; + +WITH +x AS (SELECT number AS a FROM numbers(10)), +y AS (SELECT number AS a FROM numbers(5)) +SELECT * FROM x JOIN y USING a +ORDER BY x.a; + +WITH +x AS (SELECT number AS a FROM numbers(10)), +y AS (SELECT number AS a FROM numbers(5)), +z AS (SELECT toUInt64(1) b) +SELECT * FROM x JOIN y USING a WHERE a in (SELECT * FROM z); + +WITH +x AS (SELECT number AS a FROM numbers(10)), +y AS (SELECT number AS a FROM numbers(5)), +z AS (SELECT * FROM x WHERE a % 2), +w AS (SELECT * FROM y WHERE a > 0) +SELECT * FROM x JOIN y USING a WHERE a in (SELECT * FROM z) +ORDER BY x.a; + +WITH +x AS (SELECT number AS a FROM numbers(10)), +y AS (SELECT number AS a FROM numbers(5)), +z AS (SELECT * FROM x WHERE a % 2), +w AS (SELECT * FROM y WHERE a > 0) +SELECT max(a) FROM x JOIN y USING a WHERE a in (SELECT * FROM z) +HAVING a > (SELECT min(a) FROM w); + +WITH +x AS (SELECT number AS a FROM numbers(10)), +y AS (SELECT number AS a FROM numbers(5)), +z AS (SELECT * FROM x WHERE a % 2), +w AS (SELECT * FROM y WHERE a > 0) +SELECT a FROM x JOIN y USING a WHERE a in (SELECT * FROM z) +HAVING a <= (SELECT max(a) FROM w) +ORDER BY x.a; diff --git a/tests/queries/0_stateless/01495_subqueries_in_with_statement_3.reference b/tests/queries/0_stateless/01495_subqueries_in_with_statement_3.reference new file mode 100644 index 00000000000..7cc310d9662 --- /dev/null +++ b/tests/queries/0_stateless/01495_subqueries_in_with_statement_3.reference @@ -0,0 +1,8 @@ +4999 +333 +3333 +2000 +97 1 33 +99 -3299 3399 +99 0 100 +99 -3299 3399 diff --git a/tests/queries/0_stateless/01495_subqueries_in_with_statement_3.sql b/tests/queries/0_stateless/01495_subqueries_in_with_statement_3.sql new file mode 100644 index 00000000000..36cba596d00 --- /dev/null +++ b/tests/queries/0_stateless/01495_subqueries_in_with_statement_3.sql @@ -0,0 +1,82 @@ +DROP TABLE IF EXISTS cte1; +DROP TABLE IF EXISTS cte2; + +CREATE TABLE cte1(a Int64) ENGINE=Memory; +CREATE TABLE cte2(a Int64) ENGINE=Memory; + +INSERT INTO cte1 SELECT * FROM numbers(10000); +INSERT INTO cte2 SELECT * FROM numbers(5000); + +WITH +x AS (SELECT * FROM cte1), +y AS (SELECT * FROM cte2), +z AS (SELECT * FROM x WHERE a % 2 = 1), +w AS (SELECT * FROM y WHERE a > 333) +SELECT max(a) +FROM x JOIN y USING (a) +WHERE a in (SELECT * FROM z) AND a <= (SELECT max(a) FROM w); + +WITH +x AS (SELECT * FROM cte1), +y AS (SELECT * FROM cte2), +z AS (SELECT * FROM x WHERE a % 3 = 1), +w AS (SELECT * FROM y WHERE a > 333 AND a < 1000) +SELECT count(a) +FROM x left JOIN y USING (a) +WHERE a in (SELECT * FROM z) AND a <= (SELECT max(a) FROM w); + +WITH +x AS (SELECT * FROM cte1), +y AS (SELECT * FROM cte2), +z AS (SELECT * FROM x WHERE a % 3 = 1), +w AS (SELECT * FROM y WHERE a > 333 AND a < 1000) +SELECT count(a) +FROM x left JOIN y USING (a) +WHERE a in (SELECT * FROM z); + +WITH +x AS (SELECT a-4000 a FROM cte1 WHERE cte1.a >700), +y AS (SELECT * FROM cte2), +z AS (SELECT * FROM x WHERE a % 3 = 1), +w AS (SELECT * FROM y WHERE a > 333 AND a < 1000) +SELECT count(*) +FROM x left JOIN y USING (a) +WHERE a in (SELECT * FROM z); + +WITH +x AS (SELECT a-4000 a FROM cte1 WHERE cte1.a >700), +y AS (SELECT * FROM cte2), +z AS (SELECT * FROM x WHERE a % 3 = 1), +w AS (SELECT * FROM y WHERE a > 333 AND a < 1000) +SELECT max(a), min(a), count(*) +FROM x +WHERE a in (SELECT * FROM z) AND a <100; + +WITH +x AS (SELECT a-4000 a FROM cte1 WHERE cte1.a >700), +y AS (SELECT * FROM cte2), +z AS (SELECT * FROM x WHERE a % 3 = 1), +w AS (SELECT * FROM y WHERE a > 333 AND a < 1000) +SELECT max(a), min(a), count(*) FROM x +WHERE a <100; + +WITH +x AS (SELECT a-4000 a FROM cte1 AS t WHERE cte1.a >700), +y AS (SELECT * FROM cte2), +z AS (SELECT * FROM x WHERE a % 3 = 1), +w AS (SELECT * FROM y WHERE a > 333 AND a < 1000) +SELECT max(a), min(a), count(*) +FROM y +WHERE a <100; + +WITH +x AS (SELECT a-4000 a FROM cte1 t WHERE t.a >700), +y AS (SELECT x.a a FROM x left JOIN cte1 USING (a)), +z AS (SELECT * FROM x WHERE a % 3 = 1), +w AS (SELECT * FROM y WHERE a > 333 AND a < 1000) +SELECT max(a), min(a), count(*) +FROM y +WHERE a <100; + +DROP TABLE cte1; +DROP TABLE cte2; From 4ff1dacaa0529410558a33141fade0f6fc91cc9c Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Wed, 4 Nov 2020 14:20:11 +0000 Subject: [PATCH 196/314] Fix the test case for maxMap function --- src/AggregateFunctions/AggregateFunctionSumMap.h | 1 + tests/queries/0_stateless/01280_min_map_max_map.sql | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index 33fd87c0a87..c991474d0e6 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -256,6 +256,7 @@ public: // Final step does compaction of keys that have zero values, this mutates the state auto & merged_maps = this->data(place).merged_maps; + // Remove keys which are zeros or empty. This should be enabled only for sumMap. if constexpr (compact) { for (auto it = merged_maps.cbegin(); it != merged_maps.cend();) diff --git a/tests/queries/0_stateless/01280_min_map_max_map.sql b/tests/queries/0_stateless/01280_min_map_max_map.sql index 61d4a8ee4c9..cc07998bb79 100644 --- a/tests/queries/0_stateless/01280_min_map_max_map.sql +++ b/tests/queries/0_stateless/01280_min_map_max_map.sql @@ -36,5 +36,5 @@ select maxMap(val, cnt) from values ('val Array(Enum16(\'a\'=1)), cnt Array(Int1 select minMap(val, cnt) from values ('val Array(UInt64), cnt Array(UInt64)', ([1], [0]), ([2], [0])); select maxMap(val, cnt) from values ('val Array(UInt64), cnt Array(UInt64)', ([1], [0]), ([2], [0])); select minMap(val, cnt) from values ('val Array(String), cnt Array(String)', (['A'], ['']), (['B'], [''])); -select minMap(val, cnt) from values ('val Array(String), cnt Array(String)', (['A'], ['']), (['B'], [''])); +select maxMap(val, cnt) from values ('val Array(String), cnt Array(String)', (['A'], ['']), (['B'], [''])); select sumMap(val, cnt) from values ('val Array(UInt64), cnt Array(UInt64)', ([1], [0]), ([2], [0])); From 5b9d48c71555c33b5383bebd866c2e9f4a0a11a5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 17:39:18 +0300 Subject: [PATCH 197/314] Remove garbage from docs --- docker/test/stress/README.md | 5 +- docker/test/stress/stress | 2 - docs/es/development/tests.md | 262 +--------------------------------- docs/fa/development/tests.md | 263 +---------------------------------- docs/fr/development/tests.md | 262 +--------------------------------- docs/ja/development/tests.md | 262 +--------------------------------- docs/tr/development/tests.md | 263 +---------------------------------- docs/zh/development/tests.md | 237 ------------------------------- 8 files changed, 7 insertions(+), 1549 deletions(-) mode change 100644 => 120000 docs/es/development/tests.md mode change 100644 => 120000 docs/fa/development/tests.md mode change 100644 => 120000 docs/fr/development/tests.md mode change 100644 => 120000 docs/ja/development/tests.md mode change 100644 => 120000 docs/tr/development/tests.md delete mode 100644 docs/zh/development/tests.md diff --git a/docker/test/stress/README.md b/docker/test/stress/README.md index c9b6da37b05..f747996fa2d 100644 --- a/docker/test/stress/README.md +++ b/docker/test/stress/README.md @@ -1,7 +1,6 @@ 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. +Actually it runs multiple copies of clickhouse-test (functional tests). +This allows to find problems like segmentation fault which cause shutdown of server. Usage: ``` diff --git a/docker/test/stress/stress b/docker/test/stress/stress index f29ebc55141..874dca751f3 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -68,8 +68,6 @@ if __name__ == "__main__": parser.add_argument("--test-cmd", default='/usr/bin/clickhouse-test') parser.add_argument("--skip-func-tests", default='') 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("--global-time-limit", type=int, default=3600) diff --git a/docs/es/development/tests.md b/docs/es/development/tests.md deleted file mode 100644 index ae2b61b6ba7..00000000000 --- a/docs/es/development/tests.md +++ /dev/null @@ -1,261 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 69 -toc_title: "C\xF3mo ejecutar pruebas de ClickHouse" ---- - -# Pruebas de ClickHouse {#clickhouse-testing} - -## Pruebas funcionales {#functional-tests} - -Las pruebas funcionales son las más simples y cómodas de usar. La mayoría de las características de ClickHouse se pueden probar con pruebas funcionales y son obligatorias para cada cambio en el código de ClickHouse que se puede probar de esa manera. - -Cada prueba funcional envía una o varias consultas al servidor ClickHouse en ejecución y compara el resultado con la referencia. - -Las pruebas se encuentran en `queries` directorio. Hay dos subdirectorios: `stateless` y `stateful`. Las pruebas sin estado ejecutan consultas sin datos de prueba precargados: a menudo crean pequeños conjuntos de datos sintéticos sobre la marcha, dentro de la prueba misma. Las pruebas estatales requieren datos de prueba precargados de Yandex.Métrica y no está disponible para el público en general. Tendemos a usar sólo `stateless` pruebas y evitar la adición de nuevos `stateful` prueba. - -Cada prueba puede ser de dos tipos: `.sql` y `.sh`. `.sql` test es el script SQL simple que se canaliza a `clickhouse-client --multiquery --testmode`. `.sh` test es un script que se ejecuta por sí mismo. - -Para ejecutar todas las pruebas, use `clickhouse-test` herramienta. Mira `--help` para la lista de posibles opciones. Simplemente puede ejecutar todas las pruebas o ejecutar un subconjunto de pruebas filtradas por subcadena en el nombre de la prueba: `./clickhouse-test substring`. - -La forma más sencilla de invocar pruebas funcionales es copiar `clickhouse-client` a `/usr/bin/`, ejecutar `clickhouse-server` y luego ejecutar `./clickhouse-test` de su propio directorio. - -Para agregar una nueva prueba, cree un `.sql` o `.sh` archivo en `queries/0_stateless` directorio, compruébelo manualmente y luego genere `.reference` archivo de la siguiente manera: `clickhouse-client -n --testmode < 00000_test.sql > 00000_test.reference` o `./00000_test.sh > ./00000_test.reference`. - -Las pruebas deben usar (crear, soltar, etc.) solo tablas en `test` base de datos que se supone que se crea de antemano; también las pruebas pueden usar tablas temporales. - -Si desea utilizar consultas distribuidas en pruebas funcionales, puede aprovechar `remote` función de la tabla con `127.0.0.{1..2}` direcciones para que el servidor se consulte; o puede usar clústeres de prueba predefinidos en el archivo de configuración del servidor como `test_shard_localhost`. - -Algunas pruebas están marcadas con `zookeeper`, `shard` o `long` en sus nombres. -`zookeeper` es para pruebas que están usando ZooKeeper. `shard` es para pruebas que -requiere servidor para escuchar `127.0.0.*`; `distributed` o `global` tienen el mismo -significado. `long` es para pruebas que duran un poco más de un segundo. Usted puede -deshabilitar estos grupos de pruebas utilizando `--no-zookeeper`, `--no-shard` y -`--no-long` opciones, respectivamente. - -## Bugs Conocidos {#known-bugs} - -Si conocemos algunos errores que se pueden reproducir fácilmente mediante pruebas funcionales, colocamos pruebas funcionales preparadas en `tests/queries/bugs` directorio. Estas pruebas se moverán a `tests/queries/0_stateless` cuando se corrigen errores. - -## Pruebas de integración {#integration-tests} - -Las pruebas de integración permiten probar ClickHouse en la configuración agrupada y la interacción de ClickHouse con otros servidores como MySQL, Postgres, MongoDB. Son útiles para emular divisiones de red, caídas de paquetes, etc. Estas pruebas se ejecutan bajo Docker y crean múltiples contenedores con varios software. - -Ver `tests/integration/README.md` sobre cómo ejecutar estas pruebas. - -Tenga en cuenta que la integración de ClickHouse con controladores de terceros no se ha probado. Además, actualmente no tenemos pruebas de integración con nuestros controladores JDBC y ODBC. - -## Pruebas unitarias {#unit-tests} - -Las pruebas unitarias son útiles cuando desea probar no ClickHouse como un todo, sino una sola biblioteca o clase aislada. Puede habilitar o deshabilitar la compilación de pruebas con `ENABLE_TESTS` Opción CMake. Las pruebas unitarias (y otros programas de prueba) se encuentran en `tests` subdirectorios en todo el código. Para ejecutar pruebas unitarias, escriba `ninja test`. Algunas pruebas usan `gtest`, pero algunos son solo programas que devuelven un código de salida distinto de cero en caso de fallo de prueba. - -No es necesariamente tener pruebas unitarias si el código ya está cubierto por pruebas funcionales (y las pruebas funcionales suelen ser mucho más simples de usar). - -## Pruebas de rendimiento {#performance-tests} - -Las pruebas de rendimiento permiten medir y comparar el rendimiento de alguna parte aislada de ClickHouse en consultas sintéticas. Las pruebas se encuentran en `tests/performance`. Cada prueba está representada por `.xml` archivo con la descripción del caso de prueba. Las pruebas se ejecutan con `clickhouse performance-test` herramienta (que está incrustada en `clickhouse` binario). Ver `--help` para la invocación. - -Cada prueba ejecuta una o varias consultas (posiblemente con combinaciones de parámetros) en un bucle con algunas condiciones para detener (como “maximum execution speed is not changing in three seconds”) y medir algunas métricas sobre el rendimiento de las consultas (como “maximum execution speed”). Algunas pruebas pueden contener condiciones previas en el conjunto de datos de pruebas precargado. - -Si desea mejorar el rendimiento de ClickHouse en algún escenario, y si se pueden observar mejoras en consultas simples, se recomienda encarecidamente escribir una prueba de rendimiento. Siempre tiene sentido usar `perf top` u otras herramientas de perf durante sus pruebas. - -## Herramientas de prueba y secuencias de comandos {#test-tools-and-scripts} - -Algunos programas en `tests` directorio no son pruebas preparadas, pero son herramientas de prueba. Por ejemplo, para `Lexer` hay una herramienta `src/Parsers/tests/lexer` que solo hacen la tokenización de stdin y escriben el resultado coloreado en stdout. Puede usar este tipo de herramientas como ejemplos de código y para exploración y pruebas manuales. - -También puede colocar un par de archivos `.sh` y `.reference` junto con la herramienta para ejecutarlo en alguna entrada predefinida, entonces el resultado del script se puede comparar con `.reference` file. Este tipo de pruebas no están automatizadas. - -## Pruebas diversas {#miscellaneous-tests} - -Hay pruebas para diccionarios externos ubicados en `tests/external_dictionaries` y para modelos aprendidos a máquina en `tests/external_models`. Estas pruebas no se actualizan y deben transferirse a pruebas de integración. - -Hay una prueba separada para inserciones de quórum. Esta prueba ejecuta el clúster ClickHouse en servidores separados y emula varios casos de fallas: división de red, caída de paquetes (entre nodos ClickHouse, entre ClickHouse y ZooKeeper, entre el servidor ClickHouse y el cliente, etc.), `kill -9`, `kill -STOP` y `kill -CONT` , como [Jepsen](https://aphyr.com/tags/Jepsen). A continuación, la prueba comprueba que todas las inserciones reconocidas se escribieron y todas las inserciones rechazadas no. - -La prueba de quórum fue escrita por un equipo separado antes de que ClickHouse fuera de código abierto. Este equipo ya no trabaja con ClickHouse. La prueba fue escrita accidentalmente en Java. Por estas razones, la prueba de quórum debe reescribirse y trasladarse a pruebas de integración. - -## Pruebas manuales {#manual-testing} - -Cuando desarrolla una nueva característica, es razonable probarla también manualmente. Puede hacerlo con los siguientes pasos: - -Construir ClickHouse. Ejecute ClickHouse desde el terminal: cambie el directorio a `programs/clickhouse-server` y ejecutarlo con `./clickhouse-server`. Se utilizará la configuración (`config.xml`, `users.xml` y archivos dentro de `config.d` y `users.d` directorios) desde el directorio actual de forma predeterminada. Para conectarse al servidor ClickHouse, ejecute `programs/clickhouse-client/clickhouse-client`. - -Tenga en cuenta que todas las herramientas de clickhouse (servidor, cliente, etc.) son solo enlaces simbólicos a un único binario llamado `clickhouse`. Puede encontrar este binario en `programs/clickhouse`. Todas las herramientas también se pueden invocar como `clickhouse tool` en lugar de `clickhouse-tool`. - -Alternativamente, puede instalar el paquete ClickHouse: ya sea una versión estable del repositorio de Yandex o puede crear un paquete para usted con `./release` en la raíz de fuentes de ClickHouse. Luego inicie el servidor con `sudo service clickhouse-server start` (o detener para detener el servidor). Busque registros en `/etc/clickhouse-server/clickhouse-server.log`. - -Cuando ClickHouse ya está instalado en su sistema, puede crear un nuevo `clickhouse` binario y reemplazar el binario existente: - -``` bash -$ sudo service clickhouse-server stop -$ sudo cp ./clickhouse /usr/bin/ -$ sudo service clickhouse-server start -``` - -También puede detener el servidor de clickhouse del sistema y ejecutar el suyo propio con la misma configuración pero con el registro en la terminal: - -``` bash -$ sudo service clickhouse-server stop -$ sudo -u clickhouse /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml -``` - -Ejemplo con gdb: - -``` bash -$ sudo -u clickhouse gdb --args /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml -``` - -Si el servidor de clickhouse del sistema ya se está ejecutando y no desea detenerlo, puede cambiar los números de `config.xml` (o anularlos en un archivo en `config.d` directorio), proporcione la ruta de datos adecuada y ejecútela. - -`clickhouse` binary casi no tiene dependencias y funciona en una amplia gama de distribuciones de Linux. Para probar rápidamente y sucio sus cambios en un servidor, simplemente puede `scp` su fresco construido `clickhouse` binario a su servidor y luego ejecútelo como en los ejemplos anteriores. - -## Entorno de prueba {#testing-environment} - -Antes de publicar la versión como estable, la implementamos en el entorno de prueba. El entorno de prueba es un clúster que procesa 1/39 parte de [El Yandex.Métrica](https://metrica.yandex.com/) datos. Compartimos nuestro entorno de pruebas con Yandex.Equipo de Metrica. ClickHouse se actualiza sin tiempo de inactividad sobre los datos existentes. Nos fijamos en un primer momento que los datos se procesan con éxito sin retraso de tiempo real, la replicación continúan trabajando y no hay problemas visibles para Yandex.Equipo de Metrica. La primera comprobación se puede hacer de la siguiente manera: - -``` sql -SELECT hostName() AS h, any(version()), any(uptime()), max(UTCEventTime), count() FROM remote('example01-01-{1..3}t', merge, hits) WHERE EventDate >= today() - 2 GROUP BY h ORDER BY h; -``` - -En algunos casos también implementamos en el entorno de prueba de nuestros equipos de amigos en Yandex: Market, Cloud, etc. También tenemos algunos servidores de hardware que se utilizan con fines de desarrollo. - -## Pruebas de carga {#load-testing} - -Después de implementar en el entorno de prueba, ejecutamos pruebas de carga con consultas del clúster de producción. Esto se hace manualmente. - -Asegúrese de que ha habilitado `query_log` en su clúster de producción. - -Recopilar el registro de consultas para un día o más: - -``` bash -$ clickhouse-client --query="SELECT DISTINCT query FROM system.query_log WHERE event_date = today() AND query LIKE '%ym:%' AND query NOT LIKE '%system.query_log%' AND type = 2 AND is_initial_query" > queries.tsv -``` - -Este es un ejemplo complicado. `type = 2` filtrará las consultas que se ejecutan correctamente. `query LIKE '%ym:%'` es seleccionar consultas relevantes de Yandex.Métrica. `is_initial_query` es seleccionar solo las consultas iniciadas por el cliente, no por ClickHouse (como partes del procesamiento de consultas distribuidas). - -`scp` este registro en su clúster de prueba y ejecútelo de la siguiente manera: - -``` bash -$ clickhouse benchmark --concurrency 16 < queries.tsv -``` - -(probablemente también desee especificar un `--user`) - -Luego déjalo por una noche o un fin de semana e ir a tomar un descanso. - -Usted debe comprobar que `clickhouse-server` no se bloquea, la huella de memoria está limitada y el rendimiento no se degrada con el tiempo. - -Los tiempos de ejecución de consultas precisos no se registran y no se comparan debido a la alta variabilidad de las consultas y el entorno. - -## Pruebas de construcción {#build-tests} - -Las pruebas de compilación permiten verificar que la compilación no esté rota en varias configuraciones alternativas y en algunos sistemas extranjeros. Las pruebas se encuentran en `ci` directorio. Ejecutan compilación desde la fuente dentro de Docker, Vagrant y, a veces, con `qemu-user-static` dentro de Docker. Estas pruebas están en desarrollo y las ejecuciones de pruebas no están automatizadas. - -Motivación: - -Normalmente lanzamos y ejecutamos todas las pruebas en una sola variante de compilación ClickHouse. Pero hay variantes de construcción alternativas que no se prueban a fondo. Ejemplos: - -- construir en FreeBSD; -- construir en Debian con bibliotecas de paquetes del sistema; -- construir con enlaces compartidos de bibliotecas; -- construir en la plataforma AArch64; -- construir en la plataforma PowerPc. - -Por ejemplo, construir con paquetes del sistema es una mala práctica, porque no podemos garantizar qué versión exacta de paquetes tendrá un sistema. Pero esto es realmente necesario para los mantenedores de Debian. Por esta razón, al menos tenemos que admitir esta variante de construcción. Otro ejemplo: la vinculación compartida es una fuente común de problemas, pero es necesaria para algunos entusiastas. - -Aunque no podemos ejecutar todas las pruebas en todas las variantes de compilaciones, queremos verificar al menos que varias variantes de compilación no estén rotas. Para este propósito utilizamos pruebas de construcción. - -## Pruebas de Compatibilidad de protocolos {#testing-for-protocol-compatibility} - -Cuando ampliamos el protocolo de red ClickHouse, probamos manualmente que el antiguo clickhouse-client funciona con el nuevo clickhouse-server y el nuevo clickhouse-client funciona con el antiguo clickhouse-server (simplemente ejecutando binarios de los paquetes correspondientes). - -## Ayuda del compilador {#help-from-the-compiler} - -Código principal de ClickHouse (que se encuentra en `dbms` directorio) se construye con `-Wall -Wextra -Werror` y con algunas advertencias habilitadas adicionales. Aunque estas opciones no están habilitadas para bibliotecas de terceros. - -Clang tiene advertencias aún más útiles: puedes buscarlas con `-Weverything` y elige algo para la compilación predeterminada. - -Para las compilaciones de producción, se usa gcc (todavía genera un código ligeramente más eficiente que clang). Para el desarrollo, el clang suele ser más conveniente de usar. Puede construir en su propia máquina con el modo de depuración (para ahorrar batería de su computadora portátil), pero tenga en cuenta que el compilador puede generar más advertencias con `-O3` debido a un mejor flujo de control y análisis entre procedimientos. Al construir con clang con el modo de depuración, la versión de depuración de `libc++` se utiliza que permite detectar más errores en tiempo de ejecución. - -## Desinfectantes {#sanitizers} - -**Dirección desinfectante**. -Ejecutamos pruebas funcionales y de integración bajo ASan por compromiso. - -**Valgrind (Memcheck)**. -Realizamos pruebas funcionales bajo Valgrind durante la noche. Se tarda varias horas. Actualmente hay un falso positivo conocido en `re2` biblioteca, ver [este artículo](https://research.swtch.com/sparse). - -**Desinfectante de comportamiento indefinido.** -Ejecutamos pruebas funcionales y de integración bajo ASan por compromiso. - -**Desinfectante de hilo**. -Ejecutamos pruebas funcionales bajo TSan por compromiso. Todavía no ejecutamos pruebas de integración bajo TSan por compromiso. - -**Desinfectante de memoria**. -Actualmente todavía no usamos MSan. - -**Asignador de depuración.** -Versión de depuración de `jemalloc` se utiliza para la compilación de depuración. - -## Fuzzing {#fuzzing} - -ClickHouse fuzzing se implementa tanto usando [LibFuzzer](https://llvm.org/docs/LibFuzzer.html) y consultas SQL aleatorias. -Todas las pruebas de fuzz deben realizarse con desinfectantes (Dirección y Undefined). - -LibFuzzer se usa para pruebas de fuzz aisladas del código de la biblioteca. Fuzzers se implementan como parte del código de prueba y tienen “_fuzzer” nombre postfixes. -El ejemplo de Fuzzer se puede encontrar en `src/Parsers/tests/lexer_fuzzer.cpp`. Las configuraciones, diccionarios y corpus específicos de LibFuzzer se almacenan en `tests/fuzz`. -Le recomendamos que escriba pruebas fuzz para cada funcionalidad que maneje la entrada del usuario. - -Fuzzers no se construyen de forma predeterminada. Para construir fuzzers ambos `-DENABLE_FUZZING=1` y `-DENABLE_TESTS=1` se deben establecer opciones. -Recomendamos deshabilitar Jemalloc mientras se construyen fuzzers. Configuración utilizada para integrar -Google OSS-Fuzz se puede encontrar en `docker/fuzz`. - -También usamos una prueba de fuzz simple para generar consultas SQL aleatorias y verificar que el servidor no muera al ejecutarlas. -Lo puedes encontrar en `00746_sql_fuzzy.pl`. Esta prueba debe ejecutarse de forma continua (de la noche a la mañana y más). - -## Auditoría de seguridad {#security-audit} - -La gente de Yandex Security Team hace una visión general básica de las capacidades de ClickHouse desde el punto de vista de la seguridad. - -## Analizadores estáticos {#static-analyzers} - -Corremos `PVS-Studio` por compromiso. Hemos evaluado `clang-tidy`, `Coverity`, `cppcheck`, `PVS-Studio`, `tscancode`. Encontrará instrucciones de uso en `tests/instructions/` directorio. También puedes leer [el artículo en ruso](https://habr.com/company/yandex/blog/342018/). - -Si usted usa `CLion` como IDE, puede aprovechar algunos `clang-tidy` comprueba fuera de la caja. - -## Endurecer {#hardening} - -`FORTIFY_SOURCE` se utiliza de forma predeterminada. Es casi inútil, pero todavía tiene sentido en casos raros y no lo desactivamos. - -## Estilo de código {#code-style} - -Se describen las reglas de estilo de código [aqui](https://clickhouse.tech/docs/en/development/style/). - -Para comprobar si hay algunas violaciones de estilo comunes, puede usar `utils/check-style` script. - -Para forzar el estilo adecuado de su código, puede usar `clang-format`. File `.clang-format` se encuentra en la raíz de las fuentes. Se corresponde principalmente con nuestro estilo de código real. Pero no se recomienda aplicar `clang-format` a los archivos existentes porque empeora el formato. Usted puede utilizar `clang-format-diff` herramienta que puede encontrar en el repositorio de origen clang. - -Alternativamente, puede intentar `uncrustify` herramienta para reformatear su código. La configuración está en `uncrustify.cfg` en la raíz de las fuentes. Es menos probado que `clang-format`. - -`CLion` tiene su propio formateador de código que debe ajustarse para nuestro estilo de código. - -## Pruebas Metrica B2B {#metrica-b2b-tests} - -Cada lanzamiento de ClickHouse se prueba con los motores Yandex Metrica y AppMetrica. Las pruebas y las versiones estables de ClickHouse se implementan en máquinas virtuales y se ejecutan con una copia pequeña del motor Metrica que procesa una muestra fija de datos de entrada. A continuación, los resultados de dos instancias del motor Metrica se comparan juntos. - -Estas pruebas son automatizadas por un equipo separado. Debido a la gran cantidad de piezas móviles, las pruebas fallan la mayor parte del tiempo por razones completamente no relacionadas, que son muy difíciles de descubrir. Lo más probable es que estas pruebas tengan un valor negativo para nosotros. Sin embargo, se demostró que estas pruebas son útiles en aproximadamente una o dos veces de cada cientos. - -## Cobertura de prueba {#test-coverage} - -A partir de julio de 2018, no realizamos un seguimiento de la cobertura de las pruebas. - -## Automatización de pruebas {#test-automation} - -Realizamos pruebas con el CI interno de Yandex y el sistema de automatización de trabajos llamado “Sandbox”. - -Los trabajos de compilación y las pruebas se ejecutan en Sandbox por confirmación. Los paquetes resultantes y los resultados de las pruebas se publican en GitHub y se pueden descargar mediante enlaces directos. Los artefactos se almacenan eternamente. Cuando envías una solicitud de extracción en GitHub, la etiquetamos como “can be tested” y nuestro sistema CI construirá paquetes ClickHouse (liberación, depuración, con desinfectante de direcciones, etc.) para usted. - -No usamos Travis CI debido al límite de tiempo y potencia computacional. -No usamos Jenkins. Se usó antes y ahora estamos felices de no estar usando Jenkins. - -[Artículo Original](https://clickhouse.tech/docs/en/development/tests/) diff --git a/docs/es/development/tests.md b/docs/es/development/tests.md new file mode 120000 index 00000000000..c03d36c3916 --- /dev/null +++ b/docs/es/development/tests.md @@ -0,0 +1 @@ +../../en/development/tests.md \ No newline at end of file diff --git a/docs/fa/development/tests.md b/docs/fa/development/tests.md deleted file mode 100644 index abab3230e2f..00000000000 --- a/docs/fa/development/tests.md +++ /dev/null @@ -1,262 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 69 -toc_title: "\u0646\u062D\u0648\u0647 \u0627\u062C\u0631\u0627\u06CC \u062A\u0633\u062A\ - \ \u0647\u0627\u06CC \u06A9\u0644\u06CC\u06A9 \u062E\u0627\u0646\u0647" ---- - -# تست کلیک {#clickhouse-testing} - -## تست های کاربردی {#functional-tests} - -تست های کاربردی ساده ترین و راحت برای استفاده هستند. بسیاری از ClickHouse ویژگی ها را می توان مورد آزمایش با استفاده از آزمون های عملکردی و آنها را اجباری به استفاده از برای هر تغییر در ClickHouse کد است که می تواند آزمایش می شود که در راه است. - -هر تست عملکردی یک یا چند نمایش داده شد به سرور در حال اجرا تاتر می فرستد و نتیجه را با مرجع مقایسه می کند. - -تست ها در واقع `queries` فهرست راهنما. دو زیرشاخه وجود دارد: `stateless` و `stateful`. تست های بدون تابعیت بدون هیچ گونه داده های تست پیش بارگذاری شده نمایش داده می شوند-اغلب مجموعه داده های مصنوعی کوچک را در پرواز در داخل تست خود ایجاد می کنند. تست های نفرت انگیز نیاز به داده های تست از قبل نصب شده از یاندکس.متریکا و در دسترس عموم نیست. ما تمایل به استفاده از تنها `stateless` تست ها و جلوگیری از اضافه کردن جدید `stateful` تستها - -هر تست می تواند یکی از دو نوع باشد: `.sql` و `.sh`. `.sql` تست اسکریپت ساده مربع است که به لوله کشی است `clickhouse-client --multiquery --testmode`. `.sh` تست یک اسکریپت است که به خودی خود اجرا است. - -برای اجرای تمام تست ها استفاده کنید `clickhouse-test` ابزار. نگاه کن `--help` برای لیستی از گزینه های ممکن. شما به سادگی می توانید تمام تست ها را اجرا کنید یا زیر مجموعه ای از تست های فیلتر شده توسط زیر رشته را در نام تست اجرا کنید: `./clickhouse-test substring`. - -ساده ترین راه برای فراخوانی تست های کاربردی کپی است `clickhouse-client` به `/usr/bin/` فرار کن `clickhouse-server` و سپس اجرا کنید `./clickhouse-test` از دایرکتوری خود را. - -برای اضافه کردن تست جدید, ایجاد یک `.sql` یا `.sh` پرونده در `queries/0_stateless` فهرست راهنما را به صورت دستی بررسی کنید و سپس تولید کنید `.reference` پرونده به روش زیر: `clickhouse-client -n --testmode < 00000_test.sql > 00000_test.reference` یا `./00000_test.sh > ./00000_test.reference`. - -تست باید استفاده کنید (ساختن, قطره, و غیره) تنها جداول در `test` پایگاه داده است که فرض بر این است که از قبل ایجاد می شود; همچنین تست می توانید جداول موقت استفاده. - -اگر شما می خواهید به استفاده از نمایش داده شد توزیع شده در تست های کاربردی, شما می توانید اهرم `remote` تابع جدول با `127.0.0.{1..2}` یا شما می توانید خوشه تست از پیش تعریف شده در فایل پیکربندی سرور مانند استفاده کنید `test_shard_localhost`. - -برخی از تست ها با مشخص شده اند `zookeeper`, `shard` یا `long` در نام خود. -`zookeeper` برای تست هایی است که از باغ وحش استفاده می کنند. `shard` برای تست هایی است که -نیاز به سرور برای گوش دادن `127.0.0.*`; `distributed` یا `global` همان -معنی. `long` برای تست هایی است که کمی طولانی تر اجرا می شوند که یک ثانیه. شما می توانید -غیر فعال کردن این گروه از تست با استفاده از `--no-zookeeper`, `--no-shard` و -`--no-long` گزینه, به ترتیب. - -## اشکالات شناخته شده {#known-bugs} - -اگر ما می دانیم برخی از اشکالات است که می تواند به راحتی توسط تست های کاربردی تکثیر, ما تست های عملکردی تهیه شده در `tests/queries/bugs` فهرست راهنما. این تست خواهد شد به نقل مکان کرد `tests/queries/0_stateless` هنگامی که اشکالات ثابت هستند. - -## تست های ادغام {#integration-tests} - -ادغام آزمون اجازه می دهد برای تست ClickHouse در خوشه پیکربندی و ClickHouse تعامل با سرور های دیگر مانند MySQL, Postgres, MongoDB. مفید برای تقلید انشعابات شبکه قطره بسته و غیره هستند. این تست ها تحت کارگر بارانداز اجرا و ایجاد ظروف متعدد با نرم افزار های مختلف. - -ببینید `tests/integration/README.md` در مورد چگونگی اجرای این تست. - -توجه داشته باشید که ادغام کلیک با رانندگان شخص ثالث تست نشده است. همچنین ما در حال حاضر تست های یکپارچه سازی با رانندگان جی بی سی و بی سی ما ندارد. - -## تست های واحد {#unit-tests} - -تست واحد مفید هستند که شما می خواهید برای تست نیست خانه کلیک به عنوان یک کل, اما یک کتابخانه جدا شده و یا کلاس. شما می توانید ساخت تست ها را فعال یا غیر فعال کنید `ENABLE_TESTS` گزینه کیک. تست واحد (و دیگر برنامه های تست) در واقع `tests` زیرشاخه در سراسر کد. برای اجرای تست واحد, نوع `ninja test`. برخی از تست ها استفاده می کنند `gtest`, اما برخی فقط برنامه هایی که بازگشت کد خروج غیر صفر در شکست تست. - -این لزوما به تست واحد اگر کد در حال حاضر توسط تست های کاربردی تحت پوشش (و تست های کاربردی معمولا بسیار ساده تر برای استفاده). - -## تست های عملکرد {#performance-tests} - -تست های عملکرد اجازه می دهد برای اندازه گیری و مقایسه عملکرد برخی از بخش جدا شده از خانه رعیتی در نمایش داده شد مصنوعی. تست ها در واقع `tests/performance`. هر تست توسط نمایندگی `.xml` فایل با شرح مورد تست. تست ها با اجرا `clickhouse performance-test` ابزار (که در تعبیه شده است `clickhouse` دودویی). ببینید `--help` برای نیایش. - -هر تست یک یا چند نمایش داده شد (احتمالا با ترکیبی از پارامترهای) در یک حلقه با برخی از شرایط برای توقف (مانند “maximum execution speed is not changing in three seconds”) و اندازه گیری برخی از معیارهای مورد عملکرد پرس و جو (مانند “maximum execution speed”). برخی از تست ها می توانند پیش شرط ها را در مجموعه داده های تست پیش بارگذاری شده داشته باشند. - -اگر شما می خواهید برای بهبود عملکرد تاتر در برخی از سناریو, و اگر پیشرفت را می توان در نمایش داده شد ساده مشاهده, بسیار توصیه می شود برای نوشتن یک تست عملکرد. همیشه حس می کند به استفاده از `perf top` و یا دیگر ابزار دقیق در طول تست های خود را. - -## ابزار تست و اسکریپت {#test-tools-and-scripts} - -برخی از برنامه ها در `tests` دایرکتوری تست تهیه نشده, اما ابزار تست. مثلا, برای `Lexer` یک ابزار وجود دارد `src/Parsers/tests/lexer` این فقط تقلید از استدین را انجام می دهد و نتیجه رنگی را به انحراف می نویسد. شما می توانید از این نوع ابزار به عنوان نمونه کد و برای اکتشاف و تست دستی استفاده کنید. - -شما همچنین می توانید جفت فایل قرار دهید `.sh` و `.reference` همراه با ابزار برای اجرا در برخی از ورودی از پیش تعریف شده - سپس نتیجه اسکریپت را می توان به مقایسه `.reference` پرونده. این نوع تست ها خودکار نیستند. - -## تست های متفرقه {#miscellaneous-tests} - -تست برای لغت نامه های خارجی واقع در وجود دارد `tests/external_dictionaries` و برای مدل های ماشین یاد گرفته شده در `tests/external_models`. این تست ها به روز نمی شوند و باید به تست های ادغام منتقل شوند. - -تست جداگانه برای درج حد نصاب وجود دارد. این اجرای آزمون ClickHouse خوشه در سرورهای جداگانه و شبیه سازی شکست های مختلف در موارد: شبکه تقسیم بسته رها کردن (بین ClickHouse گره بین ClickHouse و باغ وحش بین ClickHouse سرور و کلاینت ، ), `kill -9`, `kill -STOP` و `kill -CONT` مثل [جپسن](https://aphyr.com/tags/Jepsen). سپس چک تست که همه درج اذعان نوشته شده بود و همه درج رد شد. - -تست حد نصاب توسط تیم جداگانه نوشته شده بود قبل از کلیک باز منابع بود. این تیم دیگر با کلیکهاوس کار. تست به طور تصادفی در جاوا نوشته شده بود. به این دلایل, تست حد نصاب باید بازنویسی شود و به تست ادغام نقل مکان کرد. - -## تست دستی {#manual-testing} - -هنگامی که شما توسعه یک ویژگی جدید معقول نیز دستی تست است. شما می توانید این کار را با مراحل زیر انجام دهید: - -ساخت خانه کلیک. اجرای کلیک از ترمینال: تغییر دایرکتوری به `programs/clickhouse-server` و با `./clickhouse-server`. این پیکربندی استفاده کنید (`config.xml`, `users.xml` و فایل ها در `config.d` و `users.d` دایرکتوری ها) از دایرکتوری جاری به طور پیش فرض. برای اتصال به سرور کلیک اجرا کنید `programs/clickhouse-client/clickhouse-client`. - -توجه داشته باشید که تمام clickhouse ابزار (سرور مشتری و غیره) فقط symlinks به یک باینری به نام `clickhouse`. شما می توانید این دودویی در `programs/clickhouse`. همه ابزار همچنین می توانید به عنوان استناد شود `clickhouse tool` به جای `clickhouse-tool`. - -متناوبا شما می توانید بسته بندی کلیک را نصب کنید: در هر صورت انتشار پایدار از مخزن یاندکس و یا شما می توانید بسته را برای خودتان با ساخت `./release` در منابع کلیک خانه ریشه. سپس سرور را با شروع `sudo service clickhouse-server start` (یا توقف برای متوقف کردن سرور). به دنبال سیاهههای مربوط در `/etc/clickhouse-server/clickhouse-server.log`. - -هنگامی که تاتر در حال حاضر بر روی سیستم شما نصب شده, شما می توانید جدید ساخت `clickhouse` دودویی و جایگزین باینری موجود: - -``` bash -$ sudo service clickhouse-server stop -$ sudo cp ./clickhouse /usr/bin/ -$ sudo service clickhouse-server start -``` - -همچنین شما می توانید سیستم کلیک سرور را متوقف و اجرا خود را با همان پیکربندی اما با ورود به ترمینال: - -``` bash -$ sudo service clickhouse-server stop -$ sudo -u clickhouse /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml -``` - -به عنوان مثال با دیابت بارداری: - -``` bash -$ sudo -u clickhouse gdb --args /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml -``` - -اگر سیستم کلیک-سرور در حال اجرا است و شما نمی خواهید برای متوقف کردن, شما می توانید شماره پورت در خود تغییر دهید `config.xml` (یا نادیده گرفتن در یک فایل در `config.d` فهرست راهنما) مسیر داده مناسب را فراهم کرده و اجرا کنید. - -`clickhouse` دودویی تقریبا هیچ وابستگی و کار در سراسر طیف گسترده ای از توزیع های لینوکس. برای تست سریع و کثیف تغییرات خود را بر روی یک سرور, شما به سادگی می توانید `scp` تازه ساخته شده است `clickhouse` باینری به سرور شما و سپس به عنوان مثال بالا اجرا شود. - -## محیط تست {#testing-environment} - -قبل از انتشار انتشار به عنوان پایدار ما را در محیط تست استقرار. محیط تست یک خوشه است که بخشی از 1/39 را پردازش می کند [یاندکسمتریکا](https://metrica.yandex.com/) داده ها. ما محیط تست خود را با یاندکس به اشتراک می گذاریم.تیم متریکا تاتر بدون خرابی در بالای داده های موجود به روز رسانی. ما در ابتدا نگاه کنید که داده ها با موفقیت و بدون عقب مانده از زمان واقعی پردازش, تکرار ادامه کار و هیچ مشکلی برای یاندکس قابل مشاهده وجود دارد.تیم متریکا اولین چک را می توان در راه زیر انجام داد: - -``` sql -SELECT hostName() AS h, any(version()), any(uptime()), max(UTCEventTime), count() FROM remote('example01-01-{1..3}t', merge, hits) WHERE EventDate >= today() - 2 GROUP BY h ORDER BY h; -``` - -در برخی موارد ما نیز به تست محیط زیست از تیم های دوست ما در یاندکس استقرار: بازار, ابر, و غیره. همچنین در حال حاضر برخی از سرورهای سخت افزاری است که برای اهداف توسعه استفاده می شود. - -## تست بار {#load-testing} - -پس از استقرار به محیط تست ما تست بار با نمایش داده شد از خوشه تولید را اجرا کنید. این کار به صورت دستی انجام می شود. - -اطمینان حاصل کنید که شما را فعال کرده اند `query_log` در خوشه تولید خود را. - -جمع کردن گزارش پرس و جو برای یک روز یا بیشتر: - -``` bash -$ clickhouse-client --query="SELECT DISTINCT query FROM system.query_log WHERE event_date = today() AND query LIKE '%ym:%' AND query NOT LIKE '%system.query_log%' AND type = 2 AND is_initial_query" > queries.tsv -``` - -این یک مثال راه پیچیده است. `type = 2` نمایش داده شد که با موفقیت اجرا فیلتر کنید. `query LIKE '%ym:%'` است برای انتخاب نمایش داده شد مربوطه از یاندکس.متریکا `is_initial_query` است را انتخاب کنید تنها نمایش داده شد که توسط مشتری شروع, نه با کلیک خود (به عنوان بخش هایی از پردازش پرس و جو توزیع). - -`scp` این ورود به خوشه تست خود را و اجرا به شرح زیر است: - -``` bash -$ clickhouse benchmark --concurrency 16 < queries.tsv -``` - -(احتمالا شما همچنین می خواهید برای مشخص کردن یک `--user`) - -پس یه شب یا هفته ولش کن و برو استراحت کن - -شما باید بررسی کنید که `clickhouse-server` سقوط نمی کند, رد پای حافظه محدود است و عملکرد در طول زمان تنزل نمی. - -زمان اجرای پرس و جو دقیق ثبت نشده است و با توجه به تنوع بالا از نمایش داده شد و محیط زیست در مقایسه نیست. - -## ساخت تست {#build-tests} - -تست های ساخت اجازه می دهد تا بررسی کنید که ساخت در تنظیمات مختلف جایگزین و در برخی از سیستم های خارجی شکسته نمی شود. تست ها در واقع `ci` فهرست راهنما. ساخت از منبع داخل کارگر بارانداز ولگرد و گاهی با اجرا می شوند `qemu-user-static` در داخل کارگر بارانداز. این تست ها در حال توسعه هستند و تست اجرا می شود خودکار نیست. - -انگیزه: - -به طور معمول ما انتشار و اجرای تمام تست بر روی یک نوع واحد از ساخت تاتر. اما انواع ساخت جایگزین است که به طور کامل تست شده وجود دارد. مثالها: - -- ساخت در بورس; -- ساخت در دبیان با کتابخانه ها از بسته های سیستم; -- ساخت با لینک مشترک از کتابخانه ها; -- ساخت پلت فرم AArch64; -- ساخت بر روی پلت فرم پاور. - -مثلا, ساخت با بسته های سیستم عمل بد است, چرا که ما نمی تواند تضمین کند که چه نسخه دقیق از بسته های یک سیستم باید. اما این واقعا توسط نگهداری دبیان مورد نیاز است. به همین دلیل ما حداقل باید برای حمایت از این نوع ساخت. مثال دیگر: ارتباط مشترک یک منبع مشترک از مشکل است, اما برای برخی از علاقه مندان مورد نیاز است. - -هر چند ما می توانیم تمام تست در همه نوع از ایجاد اجرا کنید, ما می خواهیم برای بررسی حداقل که انواع ساخت های مختلف شکسته نمی. برای این منظور ما از تست های ساخت استفاده می کنیم. - -## تست برای سازگاری پروتکل {#testing-for-protocol-compatibility} - -هنگامی که ما گسترش ClickHouse پروتکل شبکه ما تست دستی که clickhouse-مشتری با این نسخهها کار جدید clickhouse-سرور و جدید clickhouse-مشتری با این نسخهها کار با clickhouse-سرور (به سادگی با در حال اجرا فایل های باینری از مربوطه بسته). - -## کمک از کامپایلر {#help-from-the-compiler} - -کد اصلی کلیک (که در واقع `dbms` فهرست راهنما) با ساخته شده است `-Wall -Wextra -Werror` و با برخی از هشدارهای اضافی را فعال کنید. اگر چه این گزینه ها برای کتابخانه های شخص ثالث فعال نیست. - -کلانگ هشدارهای بیشتری دارد - شما می توانید با `-Weverything` و انتخاب چیزی به طور پیش فرض ساخت. - -برای تولید ساخت, شورای همکاری خلیج فارس استفاده می شود (هنوز تولید کد کمی موثر تر از صدای جرنگ جرنگ). برای توسعه, صدای جرنگ جرنگ است که معمولا راحت تر به استفاده از. شما می توانید بر روی دستگاه خود را با حالت اشکال زدایی ساخت (برای صرفه جویی در باتری لپ تاپ خود را), اما لطفا توجه داشته باشید که کامپایلر قادر به تولید هشدارهای بیشتر با است `-O3` با توجه به جریان کنترل بهتر و تجزیه و تحلیل بین روش. هنگام ساخت با صدای جرنگ جرنگ, `libc++` به جای استفاده `libstdc++` و هنگامی که ساختمان با حالت اشکال زدایی, نسخه اشکال زدایی از `libc++` استفاده شده است که اجازه می دهد تا برای گرفتن خطاهای بیشتر در زمان اجرا. - -## Sanitizers {#sanitizers} - -**نشانی ضد عفونی کننده**. -ما تست های کاربردی و یکپارچه سازی را تحت عنوان بر اساس هر متعهد اجرا می کنیم. - -**Valgrind (Memcheck)**. -ما یک شبه تست های کاربردی را تحت ارزیابی قرار می دهیم. چند ساعت طول می کشد. در حال حاضر یک مثبت کاذب شناخته شده در وجود دارد `re2` کتابخانه را ببینید [این مقاله](https://research.swtch.com/sparse). - -**تعریف نشده رفتار ضد عفونی کننده.** -ما تست های کاربردی و یکپارچه سازی را تحت عنوان بر اساس هر متعهد اجرا می کنیم. - -**ضدعفونی کننده موضوع**. -ما تست های کاربردی تحت تسان بر اساس هر مرتکب اجرا. ما هنوز تست های ادغام تحت تسان بر اساس هر متعهد اجرا کنید. - -**ضد عفونی کننده حافظه**. -در حال حاضر ما هنوز از خانم استفاده نمی کنیم. - -**اشکال زدایی تخصیص.** -نسخه اشکال زدایی از `jemalloc` برای ساخت اشکال زدایی استفاده می شود. - -## Fuzzing {#fuzzing} - -ریش ریش شدن کلیک هر دو با استفاده از اجرا شده است [هرزه](https://llvm.org/docs/LibFuzzer.html) و تصادفی گذاشتن نمایش داده شد. -تمام تست ریش شدن باید با ضدعفونی کننده انجام شود (نشانی و تعریف نشده). - -پازل برای تست ریش ریش شدن جدا شده از کد کتابخانه استفاده می شود. طبع به عنوان بخشی از کد تست اجرا و “_fuzzer” نام پسوند. -به عنوان مثال ریش ریش شدن را می توان در یافت `src/Parsers/tests/lexer_fuzzer.cpp`. تنظیمات-پازل خاص, لغت نامه ها و جسم در ذخیره می شود `tests/fuzz`. -ما شما را تشویق به نوشتن تست ریش ریش شدن برای هر قابلیت که دسته ورودی کاربر. - -طبع به طور پیش فرض ساخته شده است. برای ساخت ریش ریش ریش ریش شدن هر دو `-DENABLE_FUZZING=1` و `-DENABLE_TESTS=1` گزینه ها باید تنظیم شود. -ما توصیه می کنیم برای غیر فعال کردن Jemalloc در حالی که ساختمان fuzzers. پیکربندی مورد استفاده برای ادغام ریش ریش شدن تاتر به -گوگل اوس فوز را می توان در یافت `docker/fuzz`. - -ما همچنین از تست ریش ریش شدن ساده برای تولید پرس و جو تصادفی ساده استفاده می کنیم و بررسی می کنیم که سرور نمی میرد. -شما می توانید این را در `00746_sql_fuzzy.pl`. این تست باید به طور مداوم اجرا شود (یک شبه و طولانی تر). - -## ممیزی امنیتی {#security-audit} - -مردم از تیم امنیتی یاندکس انجام برخی از بررسی اجمالی اساسی از قابلیت های تاتر از نقطه نظر امنیت. - -## تجزیه و تحلیل استاتیک {#static-analyzers} - -فرار میکنیم `PVS-Studio` بر اساس هر مرتکب. ما ارزیابی کرده ایم `clang-tidy`, `Coverity`, `cppcheck`, `PVS-Studio`, `tscancode`. شما دستورالعمل برای استفاده در پیدا `tests/instructions/` فهرست راهنما. همچنین شما می توانید به عنوان خوانده شده [مقاله در روسیه](https://habr.com/company/yandex/blog/342018/). - -در صورت استفاده `CLion` به عنوان محیط برنامه نویسی, شما می توانید اهرم برخی از `clang-tidy` چک از جعبه. - -## سخت شدن {#hardening} - -`FORTIFY_SOURCE` به طور پیش فرض استفاده می شود. این تقریبا بی فایده است, اما هنوز هم حس می کند در موارد نادر و ما این کار را غیر فعال کنید. - -## سبک کد {#code-style} - -قوانین سبک کد شرح داده شده است [اینجا](https://clickhouse.tech/docs/en/development/style/). - -برای بررسی برخی از نقض سبک مشترک, شما می توانید استفاده کنید `utils/check-style` خط نوشتن. - -به زور سبک مناسب از کد خود را, شما می توانید استفاده کنید `clang-format`. پرونده `.clang-format` در منابع ریشه واقع شده است. این بیشتر با سبک کد واقعی ما مطابقت دارد. اما توصیه نمی شود که اعمال شود `clang-format` به فایل های موجود چون باعث می شود قالب بندی بدتر است. شما می توانید استفاده کنید `clang-format-diff` ابزاری است که شما می توانید در مخزن منبع صدای جرنگ جرنگ پیدا. - -متناوبا شما می توانید سعی کنید `uncrustify` ابزار مجدد کد خود را. پیکربندی در `uncrustify.cfg` در منابع ریشه. این کمتر از تست شده است `clang-format`. - -`CLion` فرمت کد خود را دارد که باید برای سبک کد ما تنظیم شود. - -## تست های متریکا ب2 {#metrica-b2b-tests} - -هر ClickHouse نسخه تست شده با Yandex Metrica و AppMetrica موتورهای. تست و نسخه های پایدار از تاتر در ماشین های مجازی مستقر و اجرا با یک کپی کوچک از موتور متریکا است که پردازش نمونه ثابت از داده های ورودی. سپس نتایج حاصل از دو نمونه از موتور متریکا با هم مقایسه می شوند. - -این تست ها توسط تیم جداگانه خودکار می شوند. با توجه به تعداد زیادی از قطعات متحرک, تست شکست بیشتر از زمان به دلایل کاملا نامربوط, که بسیار دشوار است برای کشف کردن. به احتمال زیاد این تست ها ارزش منفی برای ما دارند. با این وجود این تست در حدود یک یا دو بار از صدها مفید ثابت شد. - -## پوشش تست {#test-coverage} - -تا جولای 2018 ما پوشش تست را پیگیری نمی کنیم. - -## اتوماسیون تست {#test-automation} - -ما تست ها را با سیستم اتوماسیون داخلی یاندکس اجرا می کنیم “Sandbox”. - -ساخت شغل و تست ها در گودال ماسهبازی در هر مرتکب اساس اجرا شود. نتیجه بسته ها و نتایج تست در گیتهاب منتشر شده و می تواند توسط لینک مستقیم دانلود. مصنوعات ابد ذخیره می شود. هنگامی که شما یک درخواست کشش ارسال در گیتهاب, ما برچسب به عنوان “can be tested” و سیستم سی ما خواهد بسته های تاتر ساخت (رهایی, اشکال زدایی, با نشانی ضد عفونی کننده, و غیره) برای شما. - -ما از تراویس سی به دلیل محدودیت در زمان و قدرت محاسباتی استفاده نمی کنیم. -ما از جنکینز استفاده نمیکنیم. این قبل از استفاده شد و در حال حاضر ما خوشحال ما با استفاده از جنکینز نیست. - -[مقاله اصلی](https://clickhouse.tech/docs/en/development/tests/) diff --git a/docs/fa/development/tests.md b/docs/fa/development/tests.md new file mode 120000 index 00000000000..c03d36c3916 --- /dev/null +++ b/docs/fa/development/tests.md @@ -0,0 +1 @@ +../../en/development/tests.md \ No newline at end of file diff --git a/docs/fr/development/tests.md b/docs/fr/development/tests.md deleted file mode 100644 index 5fd5cd36e2d..00000000000 --- a/docs/fr/development/tests.md +++ /dev/null @@ -1,261 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 69 -toc_title: "Comment ex\xE9cuter des Tests ClickHouse" ---- - -# ClickHouse Test {#clickhouse-testing} - -## Les Tests Fonctionnels {#functional-tests} - -Les tests fonctionnels sont les plus simples et pratiques à utiliser. La plupart des fonctionnalités de ClickHouse peuvent être testées avec des tests fonctionnels et elles sont obligatoires à utiliser pour chaque changement de code de ClickHouse qui peut être testé de cette façon. - -Chaque test fonctionnel envoie une ou plusieurs requêtes au serveur clickhouse en cours d'exécution et compare le résultat avec la référence. - -Les Tests sont situés dans `queries` répertoire. Il y a deux sous-répertoires: `stateless` et `stateful`. Les tests sans état exécutent des requêtes sans données de test préchargées - ils créent souvent de petits ensembles de données synthétiques à la volée, dans le test lui-même. Les tests avec État nécessitent des données de test préchargées de Yandex.Metrica et non disponible pour le grand public. Nous avons tendance à utiliser uniquement `stateless` tests et éviter d'ajouter de nouveaux `stateful` test. - -Chaque test peut être de deux types: `.sql` et `.sh`. `.sql` test est le script SQL simple qui est canalisé vers `clickhouse-client --multiquery --testmode`. `.sh` test est un script qui est exécuté par lui-même. - -Pour exécuter tous les tests, utilisez `clickhouse-test` outil. Regarder `--help` pour la liste des options possibles. Vous pouvez simplement exécuter tous les tests ou exécuter un sous ensemble de tests filtrés par sous chaîne dans le nom du test: `./clickhouse-test substring`. - -Le moyen le plus simple d'invoquer des tests fonctionnels est de copier `clickhouse-client` de `/usr/bin/`, exécuter `clickhouse-server` et puis exécutez `./clickhouse-test` à partir de son propre répertoire. - -Pour ajouter un nouveau test, créez un `.sql` ou `.sh` fichier dans `queries/0_stateless` répertoire, vérifiez-le manuellement, puis générez `.reference` fichier de la façon suivante: `clickhouse-client -n --testmode < 00000_test.sql > 00000_test.reference` ou `./00000_test.sh > ./00000_test.reference`. - -Les Tests doivent utiliser (create, drop, etc) uniquement des tables dans `test` base de données supposée être créée au préalable; les tests peuvent également utiliser des tables temporaires. - -Si vous souhaitez utiliser des requêtes distribuées dans les tests fonctionnels, vous pouvez tirer parti de `remote` fonction de table avec `127.0.0.{1..2}` ou vous pouvez utiliser des clusters de test prédéfinis dans le fichier de configuration du serveur comme `test_shard_localhost`. - -Certains tests sont marqués avec `zookeeper`, `shard` ou `long` en leurs noms. -`zookeeper` est pour les tests qui utilisent ZooKeeper. `shard` est pour les tests -nécessite l'écoute du serveur `127.0.0.*`; `distributed` ou `global` avoir le même -sens. `long` est pour les tests qui s'exécutent légèrement plus longtemps qu'une seconde. Vous pouvez -désactivez ces groupes de tests en utilisant `--no-zookeeper`, `--no-shard` et -`--no-long` options, respectivement. - -## Bugs Connus {#known-bugs} - -Si nous connaissons des bugs qui peuvent être facilement reproduits par des tests fonctionnels, nous plaçons des tests fonctionnels préparés dans `tests/queries/bugs` répertoire. Ces tests seront déplacés à `tests/queries/0_stateless` quand les bugs sont corrigés. - -## Les Tests D'Intégration {#integration-tests} - -Les tests d'intégration permettent de tester ClickHouse en configuration cluster et clickhouse interaction avec D'autres serveurs comme MySQL, Postgres, MongoDB. Ils sont utiles pour émuler les splits réseau, les chutes de paquets, etc. Ces tests sont exécutés sous Docker et créent plusieurs conteneurs avec divers logiciels. - -Voir `tests/integration/README.md` sur la façon d'exécuter ces tests. - -Notez que l'intégration de ClickHouse avec des pilotes tiers n'est pas testée. De plus, nous n'avons actuellement pas de tests d'intégration avec nos pilotes JDBC et ODBC. - -## Les Tests Unitaires {#unit-tests} - -Les tests unitaires sont utiles lorsque vous voulez tester non pas le ClickHouse dans son ensemble, mais une seule bibliothèque ou classe isolée. Vous pouvez activer ou désactiver la génération de tests avec `ENABLE_TESTS` Option CMake. Les tests unitaires (et autres programmes de test) sont situés dans `tests` sous-répertoires à travers le code. Pour exécuter des tests unitaires, tapez `ninja test`. Certains tests utilisent `gtest`, mais certains ne sont que des programmes qui renvoient un code de sortie non nul en cas d'échec du test. - -Ce n'est pas nécessairement d'avoir des tests unitaires si le code est déjà couvert par des tests fonctionnels (et les tests fonctionnels sont généralement beaucoup plus simples à utiliser). - -## Tests De Performance {#performance-tests} - -Les tests de Performance permettent de mesurer et de comparer les performances d'une partie isolée de ClickHouse sur des requêtes synthétiques. Les Tests sont situés à `tests/performance`. Chaque test est représenté par `.xml` fichier avec description du cas de test. Les Tests sont exécutés avec `clickhouse performance-test` outil (qui est incorporé dans `clickhouse` binaire). Voir `--help` pour l'invocation. - -Chaque essai d'exécuter une ou plusieurs requêtes (éventuellement avec des combinaisons de paramètres) dans une boucle avec certaines conditions pour l'arrêt (comme “maximum execution speed is not changing in three seconds”) et mesurer certaines mesures sur les performances de la requête (comme “maximum execution speed”). Certains tests peuvent contenir des conditions préalables sur un ensemble de données de test préchargé. - -Si vous souhaitez améliorer les performances de ClickHouse dans certains scénarios, et si des améliorations peuvent être observées sur des requêtes simples, il est fortement recommandé d'écrire un test de performance. Il est toujours logique d'utiliser `perf top` ou d'autres outils perf pendant vos tests. - -## Outils et Scripts de Test {#test-tools-and-scripts} - -Certains programmes dans `tests` directory ne sont pas des tests préparés, mais sont des outils de test. Par exemple, pour `Lexer` il est un outil `src/Parsers/tests/lexer` Cela fait juste la tokenisation de stdin et écrit le résultat colorisé dans stdout. Vous pouvez utiliser ce genre d'outils comme exemples de code et pour l'exploration et les tests manuels. - -Vous pouvez également placer une paire de fichiers `.sh` et `.reference` avec l'outil pour l'exécuter sur une entrée prédéfinie - alors le résultat du script peut être comparé à `.reference` fichier. Ce genre de tests ne sont pas automatisés. - -## Divers Tests {#miscellaneous-tests} - -Il existe des tests pour les dictionnaires externes situés à `tests/external_dictionaries` et pour machine appris modèles dans `tests/external_models`. Ces tests ne sont pas mis à jour et doivent être transférés aux tests d'intégration. - -Il y a un test séparé pour les inserts de quorum. Ce test exécute le cluster ClickHouse sur des serveurs séparés et émule divers cas d'échec: scission réseau, chute de paquets (entre les nœuds ClickHouse, entre Clickhouse et ZooKeeper, entre le serveur ClickHouse et le client, etc.), `kill -9`, `kill -STOP` et `kill -CONT` , comme [Jepsen](https://aphyr.com/tags/Jepsen). Ensuite, le test vérifie que toutes les insertions reconnues ont été écrites et que toutes les insertions rejetées ne l'ont pas été. - -Le test de Quorum a été écrit par une équipe distincte avant que ClickHouse ne soit open-source. Cette équipe ne travaille plus avec ClickHouse. Test a été écrit accidentellement en Java. Pour ces raisons, quorum test doit être réécrit et déplacé vers tests d'intégration. - -## Les Tests Manuels {#manual-testing} - -Lorsque vous développez une nouvelle fonctionnalité, il est raisonnable de tester également manuellement. Vous pouvez le faire avec les étapes suivantes: - -Construire ClickHouse. Exécuter ClickHouse à partir du terminal: changer le répertoire à `programs/clickhouse-server` et de l'exécuter avec `./clickhouse-server`. Il utilisera la configuration (`config.xml`, `users.xml` et les fichiers à l'intérieur `config.d` et `users.d` répertoires) à partir du répertoire courant par défaut. Pour vous connecter au serveur ClickHouse, exécutez `programs/clickhouse-client/clickhouse-client`. - -Notez que tous les outils clickhouse (serveur, client, etc.) ne sont que des liens symboliques vers un seul binaire nommé `clickhouse`. Vous pouvez trouver ce binaire à `programs/clickhouse`. Tous les outils peuvent également être invoquée comme `clickhouse tool` plutôt `clickhouse-tool`. - -Alternativement, vous pouvez installer le paquet ClickHouse: soit une version stable du référentiel Yandex, soit vous pouvez créer un paquet pour vous-même avec `./release` dans les sources de ClickHouse racine. Puis démarrez le serveur avec `sudo service clickhouse-server start` (ou stop pour arrêter le serveur). Rechercher des journaux à `/etc/clickhouse-server/clickhouse-server.log`. - -Lorsque ClickHouse est déjà installé sur votre système, vous pouvez créer un nouveau `clickhouse` binaire et remplacer le binaire: - -``` bash -$ sudo service clickhouse-server stop -$ sudo cp ./clickhouse /usr/bin/ -$ sudo service clickhouse-server start -``` - -Vous pouvez également arrêter system clickhouse-server et exécuter le vôtre avec la même configuration mais en vous connectant au terminal: - -``` bash -$ sudo service clickhouse-server stop -$ sudo -u clickhouse /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml -``` - -Exemple avec gdb: - -``` bash -$ sudo -u clickhouse gdb --args /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml -``` - -Si le système clickhouse-server est déjà en cours d'exécution et que vous ne voulez pas l'arrêter, vous pouvez modifier les numéros de port dans votre `config.xml` (ou de les remplacer dans un fichier `config.d` répertoire), fournissez le chemin de données approprié, et exécutez-le. - -`clickhouse` binary n'a presque aucune dépendance et fonctionne sur un large éventail de distributions Linux. Rapide et sale de tester vos modifications sur un serveur, vous pouvez simplement `scp` votre douce construite `clickhouse` binaire à votre serveur et ensuite l'exécuter comme dans les exemples ci-dessus. - -## L'Environnement De Test {#testing-environment} - -Avant de publier la version stable, nous la déployons sur l'environnement de test. L'environnement de test est un cluster processus 1/39 partie de [Yandex.Metrica](https://metrica.yandex.com/) données. Nous partageons notre environnement de test avec Yandex.Metrica de l'équipe. ClickHouse est mis à niveau sans temps d'arrêt au-dessus des données existantes. Nous regardons d'abord que les données sont traitées avec succès sans retard par rapport au temps réel, la réplication continue à fonctionner et il n'y a pas de problèmes visibles pour Yandex.Metrica de l'équipe. Première vérification peut être effectuée de la façon suivante: - -``` sql -SELECT hostName() AS h, any(version()), any(uptime()), max(UTCEventTime), count() FROM remote('example01-01-{1..3}t', merge, hits) WHERE EventDate >= today() - 2 GROUP BY h ORDER BY h; -``` - -Dans certains cas, nous déployons également à l'environnement de test de nos équipes d'amis dans Yandex: marché, Cloud, etc. Nous avons également des serveurs matériels qui sont utilisés à des fins de développement. - -## Les Tests De Charge {#load-testing} - -Après le déploiement dans l'environnement de test, nous exécutons des tests de charge avec des requêtes du cluster de production. Ceci est fait manuellement. - -Assurez-vous que vous avez activé `query_log` sur votre cluster de production. - -Recueillir le journal des requêtes pour une journée ou plus: - -``` bash -$ clickhouse-client --query="SELECT DISTINCT query FROM system.query_log WHERE event_date = today() AND query LIKE '%ym:%' AND query NOT LIKE '%system.query_log%' AND type = 2 AND is_initial_query" > queries.tsv -``` - -C'est une façon compliquée exemple. `type = 2` filtrera les requêtes exécutées avec succès. `query LIKE '%ym:%'` est de sélectionner les requêtes de Yandex.Metrica. `is_initial_query` est de sélectionner uniquement les requêtes initiées par le client, pas par ClickHouse lui-même (en tant que partie du traitement de requête distribué). - -`scp` ce journal à votre cluster de test et l'exécuter comme suit: - -``` bash -$ clickhouse benchmark --concurrency 16 < queries.tsv -``` - -(probablement vous voulez aussi spécifier un `--user`) - -Ensuite, laissez-le pour une nuit ou un week-end et allez vous reposer. - -Tu devrais vérifier ça `clickhouse-server` ne plante pas, l'empreinte mémoire est limitée et les performances ne se dégradent pas au fil du temps. - -Les délais précis d'exécution des requêtes ne sont pas enregistrés et ne sont pas comparés en raison de la grande variabilité des requêtes et de l'environnement. - -## Essais De Construction {#build-tests} - -Les tests de construction permettent de vérifier que la construction n'est pas interrompue sur diverses configurations alternatives et sur certains systèmes étrangers. Les Tests sont situés à `ci` répertoire. Ils exécutent build from source à L'intérieur de Docker, Vagrant, et parfois avec `qemu-user-static` à l'intérieur de Docker. Ces tests sont en cours de développement et les essais ne sont pas automatisées. - -Motivation: - -Normalement, nous libérons et exécutons tous les tests sur une seule variante de construction ClickHouse. Mais il existe des variantes de construction alternatives qui ne sont pas complètement testées. Exemple: - -- construire sur FreeBSD; -- construire sur Debian avec les bibliothèques des paquets système; -- construire avec des liens partagés de bibliothèques; -- construire sur la plate-forme AArch64; -- construire sur la plate-forme PowerPc. - -Par exemple, construire avec des paquets système est une mauvaise pratique, car nous ne pouvons pas garantir quelle version exacte des paquets un système aura. Mais c'est vraiment nécessaire pour les responsables Debian. Pour cette raison, nous devons au moins soutenir cette variante de construction. Un autre exemple: la liaison partagée est une source commune de problèmes, mais elle est nécessaire pour certains amateurs. - -Bien que nous ne puissions pas exécuter tous les tests sur toutes les variantes de builds, nous voulons vérifier au moins que les différentes variantes de build ne sont pas cassées. Pour cela nous utilisons les essais de construction. - -## Test de compatibilité du protocole {#testing-for-protocol-compatibility} - -Lorsque nous étendons le protocole réseau ClickHouse, nous testons manuellement que l'ancien clickhouse-client fonctionne avec le nouveau clickhouse-server et que le nouveau clickhouse-client fonctionne avec l'ancien clickhouse-server (simplement en exécutant des binaires à partir des paquets correspondants). - -## L'aide du Compilateur {#help-from-the-compiler} - -Code ClickHouse principal (qui est situé dans `dbms` annuaire) est construit avec `-Wall -Wextra -Werror` et avec quelques avertissements supplémentaires activés. Bien que ces options ne soient pas activées pour les bibliothèques tierces. - -Clang a des avertissements encore plus utiles - vous pouvez les chercher avec `-Weverything` et choisissez quelque chose à construire par défaut. - -Pour les builds de production, gcc est utilisé (il génère toujours un code légèrement plus efficace que clang). Pour le développement, clang est généralement plus pratique à utiliser. Vous pouvez construire sur votre propre machine avec le mode débogage (pour économiser la batterie de votre ordinateur portable), mais veuillez noter que le compilateur est capable de générer plus d'Avertissements avec `-O3` grâce à une meilleure analyse du flux de contrôle et de l'inter-procédure. Lors de la construction avec clang avec le mode débogage, la version de débogage de `libc++` est utilisé qui permet d'attraper plus d'erreurs à l'exécution. - -## Désinfectant {#sanitizers} - -**Désinfectant d'adresse**. -Nous exécutons des tests fonctionnels et d'intégration sous ASan sur la base de per-commit. - -**Valgrind (Memcheck)**. -Nous effectuons des tests fonctionnels sous Valgrind pendant la nuit. Cela prend plusieurs heures. Actuellement il y a un faux positif connu dans `re2` bibliothèque, consultez [cet article](https://research.swtch.com/sparse). - -**Désinfectant de comportement indéfini.** -Nous exécutons des tests fonctionnels et d'intégration sous ASan sur la base de per-commit. - -**Désinfectant pour filetage**. -Nous exécutons des tests fonctionnels sous TSan sur la base de per-commit. Nous n'exécutons toujours pas de tests D'intégration sous TSan sur la base de la validation. - -**Mémoire de désinfectant**. -Actuellement, nous n'utilisons toujours pas MSan. - -**Débogueur allocateur.** -Version de débogage de `jemalloc` est utilisé pour la construction de débogage. - -## Fuzzing {#fuzzing} - -Clickhouse fuzzing est implémenté à la fois en utilisant [libFuzzer](https://llvm.org/docs/LibFuzzer.html) et des requêtes SQL aléatoires. -Tous les tests de fuzz doivent être effectués avec des désinfectants (adresse et indéfini). - -LibFuzzer est utilisé pour les tests de fuzz isolés du code de la bibliothèque. Les Fuzzers sont implémentés dans le cadre du code de test et ont “_fuzzer” nom postfixes. -Exemple Fuzzer peut être trouvé à `src/Parsers/tests/lexer_fuzzer.cpp`. Les configs, dictionnaires et corpus spécifiques à LibFuzzer sont stockés à `tests/fuzz`. -Nous vous encourageons à écrire des tests fuzz pour chaque fonctionnalité qui gère l'entrée de l'utilisateur. - -Fuzzers ne sont pas construits par défaut. Pour construire fuzzers à la fois `-DENABLE_FUZZING=1` et `-DENABLE_TESTS=1` options doivent être définies. -Nous vous recommandons de désactiver Jemalloc lors de la construction de fuzzers. Configuration utilisée pour intégrer clickhouse fuzzing à -Google OSS-Fuzz peut être trouvé à `docker/fuzz`. - -Nous utilisons également un simple test fuzz pour générer des requêtes SQL aléatoires et vérifier que le serveur ne meurt pas en les exécutant. -Vous pouvez le trouver dans `00746_sql_fuzzy.pl`. Ce test doit être exécuté en continu (pendant la nuit et plus longtemps). - -## Audit De Sécurité {#security-audit} - -Les gens de L'équipe de sécurité Yandex font un aperçu de base des capacités de ClickHouse du point de vue de la sécurité. - -## Analyseurs Statiques {#static-analyzers} - -Nous courons `PVS-Studio` par commettre base. Nous avons évalué `clang-tidy`, `Coverity`, `cppcheck`, `PVS-Studio`, `tscancode`. Vous trouverez des instructions pour l'utilisation dans `tests/instructions/` répertoire. Aussi, vous pouvez lire [l'article en russe](https://habr.com/company/yandex/blog/342018/). - -Si vous utilisez `CLion` en tant QU'IDE, vous pouvez tirer parti de certains `clang-tidy` contrôles de la boîte. - -## Durcir {#hardening} - -`FORTIFY_SOURCE` est utilisé par défaut. C'est presque inutile, mais cela a toujours du sens dans de rares cas et nous ne le désactivons pas. - -## Code De Style {#code-style} - -Les règles de style de Code sont décrites [ici](https://clickhouse.tech/docs/en/development/style/). - -Pour vérifier certaines violations de style courantes, vous pouvez utiliser `utils/check-style` script. - -Pour forcer le style approprié de votre code, vous pouvez utiliser `clang-format`. Fichier `.clang-format` est situé à la racine des sources. Il correspond principalement à notre style de code réel. Mais il n'est pas recommandé d'appliquer `clang-format` pour les fichiers existants, car il rend le formatage pire. Vous pouvez utiliser `clang-format-diff` outil que vous pouvez trouver dans clang référentiel source. - -Alternativement vous pouvez essayer `uncrustify` outil pour reformater votre code. La Configuration est en `uncrustify.cfg` dans la racine des sources. Il est moins testé que `clang-format`. - -`CLion` a son propre formateur de code qui doit être réglé pour notre style de code. - -## Tests Metrica B2B {#metrica-b2b-tests} - -Chaque version de ClickHouse est testée avec les moteurs Yandex Metrica et AppMetrica. Les versions de test et stables de ClickHouse sont déployées sur des machines virtuelles et exécutées avec une petite copie de metrica engine qui traite un échantillon fixe de données d'entrée. Ensuite, les résultats de deux instances de metrica engine sont comparés ensemble. - -Ces tests sont automatisés par une équipe distincte. En raison du nombre élevé de pièces en mouvement, les tests échouent la plupart du temps complètement raisons, qui sont très difficiles à comprendre. Très probablement, ces tests ont une valeur négative pour nous. Néanmoins, ces tests se sont révélés utiles dans environ une ou deux fois sur des centaines. - -## La Couverture De Test {#test-coverage} - -En juillet 2018, nous ne suivons pas la couverture des tests. - -## Automatisation Des Tests {#test-automation} - -Nous exécutons des tests avec Yandex CI interne et le système d'automatisation des tâches nommé “Sandbox”. - -Les travaux de construction et les tests sont exécutés dans Sandbox sur une base de validation. Les paquets résultants et les résultats des tests sont publiés dans GitHub et peuvent être téléchargés par des liens directs. Les artefacts sont stockés éternellement. Lorsque vous envoyez une demande de tirage sur GitHub, nous l'étiquetons comme “can be tested” et notre système CI construira des paquets ClickHouse (release, debug, avec un désinfectant d'adresse, etc.) pour vous. - -Nous n'utilisons pas Travis CI en raison de la limite de temps et de puissance de calcul. -On n'utilise pas Jenkins. Il a été utilisé avant et maintenant nous sommes heureux de ne pas utiliser Jenkins. - -[Article Original](https://clickhouse.tech/docs/en/development/tests/) diff --git a/docs/fr/development/tests.md b/docs/fr/development/tests.md new file mode 120000 index 00000000000..c03d36c3916 --- /dev/null +++ b/docs/fr/development/tests.md @@ -0,0 +1 @@ +../../en/development/tests.md \ No newline at end of file diff --git a/docs/ja/development/tests.md b/docs/ja/development/tests.md deleted file mode 100644 index e41032a6b76..00000000000 --- a/docs/ja/development/tests.md +++ /dev/null @@ -1,261 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 69 -toc_title: "ClickHouse\u30C6\u30B9\u30C8\u306E\u5B9F\u884C\u65B9\u6CD5" ---- - -# ClickHouseのテスト {#clickhouse-testing} - -## 機能テスト {#functional-tests} - -機能テストは、最も簡単で使いやすいです。 ClickHouseの機能のほとんどは機能テストでテストすることができ、そのようにテストできるClickHouseコードのすべての変更に使用することが必須です。 - -各機能テストは、実行中のClickHouseサーバーに一つまたは複数のクエリを送信し、結果を参照と比較します。 - -テストは `queries` ディレクトリ。 サブディレクトリは二つあります: `stateless` と `stateful`. ステートレステストは、プリロードされたテストデータなしでクエリを実行します。 状態での検査が必要とな予圧試験データからのYandex.Metricaおよび一般に利用できない。 私たちは使用する傾向があります `stateless` テストと新しい追加を避ける `stateful` テストだ - -それぞれの試験できるの種類: `.sql` と `.sh`. `.sql` testは、パイプ処理される単純なSQLスクリプトです `clickhouse-client --multiquery --testmode`. `.sh` testは、それ自体で実行されるスクリプトです。 - -すべてのテストを実行するには、 `clickhouse-test` ツール。 見て! `--help` 可能なオプションのリスト。 できるだけ実行すべての試験または実行のサブセットの試験フィルター部分文字列の試験名: `./clickhouse-test substring`. - -機能テストを呼び出す最も簡単な方法は、コピーすることです `clickhouse-client` に `/usr/bin/`,run `clickhouse-server` そして、実行 `./clickhouse-test` 独自のディレクトリから。 - -新しいテストを追加するには、 `.sql` または `.sh` ファイル `queries/0_stateless` ディレクトリでチェックを手動でその生成 `.reference` 次の方法でファイル: `clickhouse-client -n --testmode < 00000_test.sql > 00000_test.reference` または `./00000_test.sh > ./00000_test.reference`. - -テストでは、テーブルのみを使用(create、dropなど)する必要があります `test` また、テストでは一時テーブルを使用することもできます。 - -機能テストで分散クエリを使用する場合は、以下を利用できます `remote` テーブル関数 `127.0.0.{1..2}` または、サーバー設定ファイルで次のように定義済みのテストクラスタを使用できます `test_shard_localhost`. - -いくつかのテストには `zookeeper`, `shard` または `long` 彼らの名前で。 -`zookeeper` ZooKeeperを使用しているテスト用です。 `shard` そのテストのためです -サーバーにリッスンが必要 `127.0.0.*`; `distributed` または `global` 同じを持っている -意味だ `long` 少し長く実行されるテストのためのものです。 あなたはできる -disableこれらのグループの試験を使用 `--no-zookeeper`, `--no-shard` と -`--no-long` オプション、それぞれ。 - -## 既知のバグ {#known-bugs} - -機能テストで簡単に再現できるいくつかのバグがわかっている場合は、準備された機能テストを `tests/queries/bugs` ディレクトリ。 これらのテストは `tests/queries/0_stateless` バグが修正されたとき。 - -## 統合テスト {#integration-tests} - -統合テストでは、クラスター化された構成でClickHouseをテストし、Mysql、Postgres、MongoDBなどの他のサーバーとClickHouseの相互作用をテストできます。 これらをエミュレートするネットワーク分割、パケットの落下など。 これらの試験する方向に作用しDockerを複数の容器を様々なソフトウェアです。 - -見る `tests/integration/README.md` これらのテストを実行する方法について。 - -この統合ClickHouse第三者によるドライバーではない。 また、現在、JDBCおよびODBCドライバとの統合テストはありません。 - -## 単体テスト {#unit-tests} - -単体テストは、ClickHouse全体ではなく、単一の孤立したライブラリまたはクラスをテストする場合に便利です。 テストのビルドを有効または無効にするには `ENABLE_TESTS` CMakeオプション。 単体テスト(およびその他のテストプログラム)は `tests` コード全体のサブディレクトリ。 単体テストを実行するには、 `ninja test`. 一部のテストでは `gtest` しかし、いくつかは、テストの失敗でゼロ以外の終了コードを返すプログラムです。 - -コードがすでに機能テストでカバーされている場合は、必ずしも単体テストを持つとは限りません(機能テストは通常ははるかに簡単です)。 - -## 性能テスト {#performance-tests} - -パフォーマ テストは `tests/performance`. それぞれの試験に代表される `.xml` テストケースの説明を持つファイル。 テストは以下で実行されます `clickhouse performance-test` ツール(埋め込まれている `clickhouse` バイナリ)。 見る `--help` 呼び出し用。 - -それぞれの試験実行または複数のクエリ(このパラメータの組み合わせ)のループ条件のための停止など “maximum execution speed is not changing in three seconds” 測定一部の指標につクエリの性能など “maximum execution speed”). いくつかの試験を含むことができ前提条件に予圧試験データを得る。 - -いくつかのシナリオでClickHouseのパフォーマンスを向上させたい場合や、単純なクエリで改善が見られる場合は、パフォーマンステストを作成することを強 いう意味があるのに使用 `perf top` またはあなたのテストの間の他のperf用具。 - -## テストツールとスクリプ {#test-tools-and-scripts} - -一部のプログラム `tests` ディレク 例えば、 `Lexer` ツールがあります `src/Parsers/tests/lexer` それはstdinのトークン化を行い、色付けされた結果をstdoutに書き込みます。 これらの種類のツールは、コード例として、また探索と手動テストに使用できます。 - -でも一対のファイル `.sh` と `.reference` いくつかの事前定義された入力でそれを実行するためのツールと一緒に-その後、スクリプトの結果は `.reference` ファイル これらの種類のテストは自動化されていません。 - -## その他のテスト {#miscellaneous-tests} - -外部辞書のテストは次の場所にあります `tests/external_dictionaries` そして機械学んだモデルのために `tests/external_models`. これらのテストは更新されず、統合テストに転送する必要があります。 - -クォーラム挿入には別のテストがあります。 このテストでは、ネットワーク分割、パケットドロップ(ClickHouseノード間、ClickHouseとZooKeeper間、ClickHouseサーバーとクライアント間など)など、さまざまな障害ケースをエミュレートします。), `kill -9`, `kill -STOP` と `kill -CONT` 例えば [ジェプセン](https://aphyr.com/tags/Jepsen). その後、試験チェックすべての認識を挿入したすべて拒否された挿入しました。 - -定足数を緩和試験の筆に別々のチーム前ClickHouseしたオープン達した. このチームはClickHouseでは動作しなくなりました。 テストは誤ってJavaで書かれました。 これらのことから、決議の定足数テストを書き換え及び移転統合。 - -## 手動テスト {#manual-testing} - -新しい機能を開発するときは、手動でもテストするのが妥当です。 これを行うには、次の手順を実行します: - -ClickHouseを構築します。 ターミナルからClickHouseを実行します。 `programs/clickhouse-server` そして、それを実行します `./clickhouse-server`. それは構成を使用します (`config.xml`, `users.xml` そして内のファイル `config.d` と `users.d` ディレクトリ)から、現在のディレクトリがデフォルトです。 ClickHouseサーバーに接続するには、以下を実行します `programs/clickhouse-client/clickhouse-client`. - -これらのclickhouseツール(サーバ、クライアント、などだそうでsymlinks単一のバイナリ名 `clickhouse`. このバイナリは `programs/clickhouse`. すべてのツ `clickhouse tool` 代わりに `clickhouse-tool`. - -またインストールすることができClickHouseパッケージは安定したリリースからのYandexリポジトリあるいはすることで作ることができるパッケージで `./release` ClickHouseソースルートで. 次に、サーバーを起動します `sudo service clickhouse-server start` (または停止してサーバーを停止します)。 ログを探す `/etc/clickhouse-server/clickhouse-server.log`. - -時ClickHouseでに既にインストールされているシステムを構築できる新しい `clickhouse` 既存のバイナリを置き換えます: - -``` bash -$ sudo service clickhouse-server stop -$ sudo cp ./clickhouse /usr/bin/ -$ sudo service clickhouse-server start -``` - -また、システムclickhouse-serverを停止し、同じ構成ではなく端末にログインして独自のものを実行することもできます: - -``` bash -$ sudo service clickhouse-server stop -$ sudo -u clickhouse /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml -``` - -Gdbの例: - -``` bash -$ sudo -u clickhouse gdb --args /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml -``` - -システムclickhouse-serverがすでに実行されていて、それを停止したくない場合は、次のポート番号を変更できます `config.xml` (または、ファイル内でそれらを上書きする `config.d` ディレクトリ)、適切なデータパスを提供し、それを実行します。 - -`clickhouse` バイナリーはほとんどない依存関係の作品を広い範囲のLinuxディストリビューション. サーバー上で変更を迅速かつ汚いテストするには、次のことができます `scp` あなたの新鮮な構築 `clickhouse` あなたのサーバーにバイナリし、上記の例のように実行します。 - -## テスト環境 {#testing-environment} - -リリースを安定版として公開する前に、テスト環境に展開します。 テスト環境は1/39の部分を処理する集りです [Yandex.メトリカ](https://metrica.yandex.com/) データ テスト環境をYandexと共有しています。メトリカ-チーム ClickHouseは既存のデータの上にダウンタイムなしで改善される。 私たちは、データがリアルタイムから遅れることなく正常に処理され、複製が動作し続け、Yandexに見える問題はないことを最初に見ています。メトリカ-チーム 最初のチェックは、次の方法で行うことができます: - -``` sql -SELECT hostName() AS h, any(version()), any(uptime()), max(UTCEventTime), count() FROM remote('example01-01-{1..3}t', merge, hits) WHERE EventDate >= today() - 2 GROUP BY h ORDER BY h; -``` - -市場、クラウドなど:いくつかのケースでは、我々はまた、Yandexの中で私たちの友人チームのテスト環境に展開します また、開発目的で使用されるハードウェアサーバーもあります。 - -## 負荷テスト {#load-testing} - -後の展開を試験環境を実行負荷テストクエリから生産ます。 これは手動で行われます。 - -有効にしていることを確認します `query_log` 運用クラスター上。 - -一日以上のクエリログを収集する: - -``` bash -$ clickhouse-client --query="SELECT DISTINCT query FROM system.query_log WHERE event_date = today() AND query LIKE '%ym:%' AND query NOT LIKE '%system.query_log%' AND type = 2 AND is_initial_query" > queries.tsv -``` - -これは複雑な例です。 `type = 2` 正常に実行されたクエリをフィルタ処理します。 `query LIKE '%ym:%'` Yandexから関連するクエリを選択することです。メトリカ `is_initial_query` ClickHouse自体ではなく、クライアントによって開始されたクエリのみを選択することです(分散クエリ処理の一部として)。 - -`scp` このログをテストクラスタに記録し、次のように実行します: - -``` bash -$ clickhouse benchmark --concurrency 16 < queries.tsv -``` - -(おそらくあなたはまた、 `--user`) - -それから夜または週末のためにそれを残し、残りを取る行きなさい。 - -きることを確認 `clickhouse-server` なクラッシュメモリのフットプリントは有界性なつ品位を傷つける。 - -クエリと環境の変動が大きいため、正確なクエリ実行タイミングは記録されず、比較されません。 - -## ビルドテスト {#build-tests} - -構築を試験できることを確認の構築においても様々な代替構成されており、外国のシステム。 テストは `ci` ディレクトリ。 Docker、Vagrant、時には以下のようなソースからビルドを実行します `qemu-user-static` ドッカー内部。 これらのテストは開発中であり、テストの実行は自動化されません。 - -動機: - -通常、ClickHouse buildの単一のバリアントですべてのテストをリリースして実行します。 しかし、徹底的にテストされていない別のビルド変種があります。 例: - -- FreeBSD上でビルド; -- をDebianを対象として図書館システムのパッケージ; -- ライブラリの共有リンクでビルド; -- AArch64プラットフォ; -- PowerPcプラットフォーム上で構築。 - -たとえば、システムパッケージを使用したビルドは悪い習慣です。 しかし、これは本当にDebianメンテナに必要です。 このため、少なくともこのビルドの変種をサポートする必要があります。 別の例:共有リンクは一般的な問題の原因ですが、一部の愛好家にとって必要です。 - -ができませんので実行した全試験はすべての変異体を構築し、チェックしたい少なくとも上記に記載された各種の構築異な破となりました。 この目的のためにビルドテストを使用します。 - -## プロトコル互換性のテスト {#testing-for-protocol-compatibility} - -ClickHouse network protocolを拡張すると、古いclickhouse-clientが新しいclickhouse-serverで動作し、新しいclickhouse-clientが古いclickhouse-serverで動作することを手動でテストします(対応するパッケージからバイナリを - -## コンパイラからのヘルプ {#help-from-the-compiler} - -メインクリックハウスコード(にある `dbms` ディレクトリ)は `-Wall -Wextra -Werror` そして、いくつかの追加の有効な警告と。 これらのオプションは有効になっていないためにサードパーティーのライブラリ. - -Clangにはさらに便利な警告があります。 `-Weverything` デフォルトのビルドに何かを選ぶ。 - -本番ビルドでは、gccが使用されます(clangよりもやや効率的なコードが生成されます)。 開発のために、clangは通常、使用する方が便利です。 あなたは(あなたのラップトップのバッテリーを節約するために)デバッグモードで自分のマシン上で構築することができますが、コンパイラがでより `-O3` よりよい制御フローおよびinter-procedure分析が原因で。 Clangでビルドする場合, `libc++` の代わりに使用されます。 `libstdc++` そして、デバッグモードでビルドするとき、 `libc++` 使用可能にするにはより誤差があります。. - -## サニタイザー {#sanitizers} - -**アドレスsanitizer**. -私たちは、コミットごとにASanの下で機能テストと統合テストを実行します。 - -**ヴァルグリンド(曖昧さ回避)**. -私たちは一晩Valgrindの下で機能テストを実行します。 数時間かかります。 現在知られている偽陽性があります `re2` 図書館、参照 [この記事](https://research.swtch.com/sparse). - -**未定義の動作のサニタイザー。** -私たちは、コミットごとにASanの下で機能テストと統合テストを実行します。 - -**糸のsanitizer**. -私たちは、コミットごとにTSanの下で機能テストを実行します。 コミットごとにTSanの下で統合テストを実行することはまだありません。 - -**メモリサニタイザー**. -現在、我々はまだMSanを使用していません。 - -**デバッグアロケータ。** -デバッグバージョン `jemalloc` デバッグビルドに使用されます。 - -## ファジング {#fuzzing} - -ClickHouseファジングは、両方を使用して実装されます [libFuzzer](https://llvm.org/docs/LibFuzzer.html) とランダムSQLクエリ。 -すべてのファズテストは、サニタイザー(アドレスと未定義)で実行する必要があります。 - -LibFuzzerは、ライブラリコードの分離ファズテストに使用されます。 ファザーはテストコードの一部として実装され “_fuzzer” 名前の接尾辞。 -Fuzzerの例はで見つけることができます `src/Parsers/tests/lexer_fuzzer.cpp`. LibFuzzer固有の設定、辞書、およびコーパスは次の場所に格納されます `tests/fuzz`. -ご協力をお願いいたし書きファズ試験べての機能を取り扱うユーザー入力します。 - -ファザーはデフォルトではビルドされません。 両方のファザーを構築するには `-DENABLE_FUZZING=1` と `-DENABLE_TESTS=1` 選択は置かれるべきである。 -ファザーのビルド中にJemallocを無効にすることをお勧めします。 ClickHouseファジングを統合するために使用される設定 -Google OSS-Fuzzは次の場所にあります `docker/fuzz`. - -また簡単なファズ試験をランダムなSQLクエリーやことを確認するにはサーバーにな金型を実行します。 -それを見つけることができる `00746_sql_fuzzy.pl`. このテストは、継続的に実行する必要があります(一晩と長い)。 - -## セキュリティ監査 {#security-audit} - -人からのYandexセキュリティチームはいくつかの基本的な概要ClickHouse力からのセキュリティの観点から. - -## 静的アナライザ {#static-analyzers} - -私たちは走る `PVS-Studio` コミットごと。 私達は評価しました `clang-tidy`, `Coverity`, `cppcheck`, `PVS-Studio`, `tscancode`. 使用のための指示をで見つけます `tests/instructions/` ディレクトリ。 また読むことができます [ロシア語の記事](https://habr.com/company/yandex/blog/342018/). - -を使用する場合 `CLion` IDEとして、いくつかを活用できます `clang-tidy` 箱から出してチェックします。 - -## 硬化 {#hardening} - -`FORTIFY_SOURCE` デフォルトで使用されます。 それはほとんど役に立たないですが、まれに理にかなっており、それを無効にしません。 - -## コードスタイル {#code-style} - -コードのスタイルのルールを記述 [ここに](https://clickhouse.tech/docs/en/development/style/). - -チェックのための、共通したスタイル違反、利用できる `utils/check-style` スクリプト - -コードの適切なスタイルを強制するには、次のようにします `clang-format`. ファイル `.clang-format` ソースルートにあります。 実際のコードスタイルにほとんど対応しています。 しかし、適用することはお勧めしません `clang-format` 既存のファイルへの書式設定が悪化するためです。 以下を使用できます `clang-format-diff` clangソースリポジトリで見つけることができるツール。 - -あるいは、 `uncrustify` コードを再フォーマットするツール。 設定は次のとおりです `uncrustify.cfg` ソースルートで。 それはより少なくテストさ `clang-format`. - -`CLion` 独自のコードをフォーマッタしていると見ることができる調整のためのコードです。 - -## Metrica B2Bテスト {#metrica-b2b-tests} - -各ClickHouseリリースはYandex MetricaとAppMetricaエンジンでテストされます。 ClickHouseのテスト版と安定版はVmにデプロイされ、入力データの固定サンプルを処理するMetrica engineの小さなコピーで実行されます。 次に,Metricaエンジンの二つのインスタンスの結果を比較した。 - -これらの試験により自動化されており、別のチームです。 可動部分の高い数が原因で、テストは把握し非常ににくい完全に無関係な理由によって失敗ほとんどの時間です。 がこれらの試験は負の値です。 しかしこれらの試験することが明らかとなったが有用である一又は二倍の数百名 - -## テスト範囲 {#test-coverage} - -2018年現在、テストカバーは行っていない。 - -## テスト自動化 {#test-automation} - -Yandex内部CIとジョブ自動化システムという名前のテストを実行します “Sandbox”. - -ビルドジョブとテストは、コミットごとにSandboxで実行されます。 結果のパッケージとテスト結果はGitHubに公開され、直接リンクでダウンロードできます。 成果物は永遠に保存されます。 GitHubでプルリクエストを送信すると、次のようにタグ付けします “can be tested” そして私達のCIシステムはあなたのためのClickHouseのパッケージ(住所sanitizerの解放、デバッグ、等)を造ります。 - -時間と計算能力の限界のため、Travis CIは使用しません。 -ジェンキンスは使わない 以前は使用されていましたが、今はJenkinsを使用していません。 - -[元の記事](https://clickhouse.tech/docs/en/development/tests/) diff --git a/docs/ja/development/tests.md b/docs/ja/development/tests.md new file mode 120000 index 00000000000..c03d36c3916 --- /dev/null +++ b/docs/ja/development/tests.md @@ -0,0 +1 @@ +../../en/development/tests.md \ No newline at end of file diff --git a/docs/tr/development/tests.md b/docs/tr/development/tests.md deleted file mode 100644 index a0766e54ae7..00000000000 --- a/docs/tr/development/tests.md +++ /dev/null @@ -1,262 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 69 -toc_title: "ClickHouse testleri nas\u0131l \xE7al\u0131\u015Ft\u0131r\u0131l\u0131\ - r" ---- - -# ClickHouse Testi {#clickhouse-testing} - -## Fonksiyonel Testler {#functional-tests} - -Fonksiyonel testler en basit ve kullanımı kolay olanlardır. ClickHouse özelliklerinin çoğu fonksiyonel testlerle test edilebilir ve bu şekilde test edilebilecek ClickHouse kodundaki her değişiklik için kullanılması zorunludur. - -Her işlevsel test, çalışan ClickHouse sunucusuna bir veya birden çok sorgu gönderir ve sonucu referansla karşılaştırır. - -Testler bulunur `queries` dizin. İki alt dizin var: `stateless` ve `stateful`. Durumsuz testler, önceden yüklenmiş test verileri olmadan sorguları çalıştırır - genellikle testin kendisinde anında küçük sentetik veri kümeleri oluştururlar. Durum bilgisi testleri, Yandex'ten önceden yüklenmiş test verileri gerektirir.Metrica ve halka açık değil. Biz sadece kullanmak eğilimindedir `stateless` testler ve yeni eklemekten kaçının `stateful` testler. - -Her test iki tipten biri olabilir: `.sql` ve `.sh`. `.sql` test için borulu basit SQL komut dosyasıdır `clickhouse-client --multiquery --testmode`. `.sh` test kendisi tarafından çalıştırılan bir komut dosyasıdır. - -Tüm testleri çalıştırmak için şunları kullanın `clickhouse-test` aracı. Bak `--help` Olası seçeneklerin listesi için. Sadece tüm testleri çalıştırmak veya test adı alt dize tarafından süzülmüş testlerin alt kümesini çalıştırabilirsiniz: `./clickhouse-test substring`. - -Fonksiyonel testleri çağırmanın en basit yolu kopyalamaktır `clickhouse-client` -e doğru `/usr/bin/`, çalıştırmak `clickhouse-server` ve sonra koş `./clickhouse-test` kendi dizininden. - -Yeni test eklemek için, bir `.sql` veya `.sh` dosya içinde `queries/0_stateless` dizin, elle kontrol edin ve sonra oluşturun `.reference` aşağıdaki şekilde dosya: `clickhouse-client -n --testmode < 00000_test.sql > 00000_test.reference` veya `./00000_test.sh > ./00000_test.reference`. - -Testler yalnızca tabloları (create, drop, vb.) kullanmalıdır `test` önceden oluşturulduğu varsayılır veritabanı; ayrıca testler geçici tablolar kullanabilirsiniz. - -İşlevsel testlerde dağıtılmış sorgular kullanmak istiyorsanız, kaldıraç `remote` tablo fonksiyonu ile `127.0.0.{1..2}` sunucunun kendisini sorgulaması için adresler; veya sunucu yapılandırma dosyasında önceden tanımlanmış test kümelerini kullanabilirsiniz `test_shard_localhost`. - -Bazı testler ile işaretlenir `zookeeper`, `shard` veya `long` kendi adlarına. -`zookeeper` ZooKeeper kullanan testler içindir. `shard` testler içindir -dinlemek için sunucu gerektirir `127.0.0.*`; `distributed` veya `global` aynı var -anlama. `long` bir saniye biraz daha uzun süren testler içindir. Yapabilirsin -kullanarak bu test gruplarını devre dışı bırakın `--no-zookeeper`, `--no-shard` ve -`--no-long` sırasıyla seçenekler. - -## Bilinen Hatalar {#known-bugs} - -Fonksiyonel testlerle kolayca çoğaltılabilen bazı hatalar biliyorsak, hazırlanmış fonksiyonel testleri `tests/queries/bugs` dizin. Bu testler taşınacaktır `tests/queries/0_stateless` hatalar düzeltildiğinde. - -## Entegrasyon Testleri {#integration-tests} - -Entegrasyon testleri, kümelenmiş konfigürasyonda Clickhouse'u ve MySQL, Postgres, MongoDB gibi diğer sunucularla ClickHouse etkileşimini test etmeyi sağlar. Ağ bölmelerini, paket damlalarını vb. taklit etmek için kullanışlıdırlar. Bu testler Docker altında çalıştırılır ve çeşitli yazılımlarla birden fazla konteyner oluşturur. - -Görmek `tests/integration/README.md` bu testlerin nasıl çalıştırılacağı hakkında. - -Clickhouse'un üçüncü taraf sürücülerle entegrasyonunun sınanmadığını unutmayın. Ayrıca şu anda JDBC ve ODBC sürücülerimizle entegrasyon testlerimiz yok. - -## Ünite Testleri {#unit-tests} - -Birim testleri, Clickhouse'u bir bütün olarak değil, tek bir yalıtılmış kitaplık veya sınıfı test etmek istediğinizde kullanışlıdır. Etkinleştirebilir veya devre dışı bırakma ile testlerin yapı `ENABLE_TESTS` Cmake seçeneği. Birim testleri (ve diğer test programları) bulunur `tests` kodun alt dizinleri. Birim testlerini çalıştırmak için şunları yazın `ninja test`. Bazı testler kullanın `gtest`, ancak bazıları test başarısızlığında sıfır olmayan çıkış kodunu döndüren programlardır. - -Kodun zaten işlevsel testler tarafından kapsanması durumunda birim testlerine sahip olmak zorunlu değildir (ve işlevsel testler genellikle kullanımı çok daha basittir). - -## Performans Testleri {#performance-tests} - -Performans testleri ölçmek ve sentetik sorguları ClickHouse bazı izole kısmının performansını karşılaştırmak için izin verir. Testler bulunur `tests/performance`. Her test ile temsil edilir `.xml` test durumunun açıklaması ile dosya. Testler ile çalıştırılır `clickhouse performance-test` Aracı (Bu gömülü `clickhouse` ikilik). Görmek `--help` çağırma için. - -Her test, durdurma için bazı koşullarla (örneğin, bir döngüde bir veya birden fazla sorgu (muhtemelen parametre kombinasyonlarıyla) çalıştırır “maximum execution speed is not changing in three seconds”) ve sorgu performansı ile ilgili bazı metrikleri ölçün (örneğin “maximum execution speed”). Bazı testler önceden yüklenmiş test veri kümesinde Önkoşullar içerebilir. - -Bazı senaryoda Clickhouse'un performansını artırmak istiyorsanız ve basit sorgularda iyileştirmeler gözlemlenebiliyorsa, bir performans testi yazmanız önerilir. Her zaman kullanmak mantıklı `perf top` testleriniz sırasında veya diğer perf araçları. - -## Test araçları ve komut dosyaları {#test-tools-and-scripts} - -Bazı programlar `tests` dizin testleri hazırlanmış değil, ancak test araçlarıdır. Örneğin, için `Lexer` bir araç var `src/Parsers/tests/lexer` bu sadece stdin'in tokenizasyonunu yapar ve renklendirilmiş sonucu stdout'a yazar. Bu tür araçları kod örnekleri olarak ve keşif ve manuel test için kullanabilirsiniz. - -Ayrıca Çift Dosya yerleştirebilirsiniz `.sh` ve `.reference` aracı ile birlikte bazı önceden tanımlanmış giriş üzerinde çalıştırmak için-daha sonra komut sonucu karşılaştırılabilir `.reference` Dosya. Bu tür testler otomatik değildir. - -## Çeşitli Testler {#miscellaneous-tests} - -Bulunan dış sözlükler için testler vardır `tests/external_dictionaries` ve makine öğrenilen modeller için `tests/external_models`. Bu testler güncelleştirilmez ve tümleştirme testlerine aktarılmalıdır. - -Çekirdek ekler için ayrı bir test var. Bu test, ayrı sunucularda ClickHouse kümesini çalıştırır ve çeşitli arıza durumlarını taklit eder: ağ bölünmesi, paket bırakma (ClickHouse düğümleri arasında, ClickHouse ve ZooKeeper arasında, ClickHouse sunucusu ve istemci arasında, vb.), `kill -9`, `kill -STOP` ve `kill -CONT` , istemek [Jepsen](https://aphyr.com/tags/Jepsen). Daha sonra test, kabul edilen tüm eklerin yazıldığını ve reddedilen tüm eklerin olmadığını kontrol eder. - -Clickhouse açık kaynaklı önce çekirdek testi ayrı ekip tarafından yazılmıştır. Bu takım artık ClickHouse ile çalışmıyor. Test yanlışlıkla Java ile yazılmıştır. Bu nedenlerden dolayı, çekirdek testi yeniden yazılmalı ve entegrasyon testlerine taşınmalıdır. - -## Manuel Test {#manual-testing} - -Yeni bir özellik geliştirdiğinizde, el ile de test etmek mantıklıdır. Bunu aşağıdaki adımlarla yapabilirsiniz: - -ClickHouse Oluşturun. Terminalden Clickhouse'u çalıştırın: dizini değiştir `programs/clickhouse-server` ve ile çalıştırın `./clickhouse-server`. Bu yapılandırma kullanacak (`config.xml`, `users.xml` ve içindeki dosyalar `config.d` ve `users.d` dizinler) geçerli dizinden varsayılan olarak. ClickHouse sunucusuna bağlanmak için, çalıştırın `programs/clickhouse-client/clickhouse-client`. - -Tüm clickhouse araçlarının (sunucu, istemci, vb.) sadece tek bir ikili için symlinks olduğunu unutmayın `clickhouse`. Bu ikili bulabilirsiniz `programs/clickhouse`. Tüm araçlar olarak da çağrılabilir `clickhouse tool` yerine `clickhouse-tool`. - -Alternatif olarak ClickHouse paketini yükleyebilirsiniz: Yandex deposundan kararlı sürüm veya kendiniz için paket oluşturabilirsiniz `./release` ClickHouse kaynakları kökünde. Ardından sunucuyu şu şekilde başlatın `sudo service clickhouse-server start` (veya sunucuyu durdurmak için durdurun). Günlükleri arayın `/etc/clickhouse-server/clickhouse-server.log`. - -ClickHouse sisteminizde zaten yüklü olduğunda, yeni bir `clickhouse` ikili ve mevcut ikili değiştirin: - -``` bash -$ sudo service clickhouse-server stop -$ sudo cp ./clickhouse /usr/bin/ -$ sudo service clickhouse-server start -``` - -Ayrıca sistem clickhouse-server durdurmak ve aynı yapılandırma ile ancak terminale günlüğü ile kendi çalıştırabilirsiniz: - -``` bash -$ sudo service clickhouse-server stop -$ sudo -u clickhouse /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml -``` - -Gdb ile örnek: - -``` bash -$ sudo -u clickhouse gdb --args /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml -``` - -Sistem clickhouse-sunucu zaten çalışıyorsa ve bunu durdurmak istemiyorsanız, sizin port numaralarını değiştirebilirsiniz `config.xml` (veya bunları bir dosyada geçersiz kılma `config.d` dizin), uygun veri yolu sağlayın ve çalıştırın. - -`clickhouse` ikili neredeyse hiçbir bağımlılıkları vardır ve Linux dağıtımları geniş genelinde çalışır. Hızlı ve kirli bir sunucuda değişikliklerinizi test etmek için, sadece yapabilirsiniz `scp` taze inşa `clickhouse` sunucunuza ikili ve daha sonra yukarıdaki örneklerde olduğu gibi çalıştırın. - -## Test Ortamı {#testing-environment} - -Kararlı olarak yayınlamadan önce test ortamında dağıtın. Test ortamı, 1/39 bölümünü işleyen bir kümedir [Üye.Metrica](https://metrica.yandex.com/) veriler. Test ortamımızı Yandex ile paylaşıyoruz.Metrica takımı. ClickHouse mevcut verilerin üstünde kesinti olmadan yükseltilir. İlk önce verilerin gerçek zamanlı olarak gecikmeden başarıyla işlendiğine bakıyoruz, çoğaltma çalışmaya devam ediyor ve Yandex tarafından görülebilen herhangi bir sorun yok.Metrica takımı. İlk kontrol aşağıdaki şekilde yapılabilir: - -``` sql -SELECT hostName() AS h, any(version()), any(uptime()), max(UTCEventTime), count() FROM remote('example01-01-{1..3}t', merge, hits) WHERE EventDate >= today() - 2 GROUP BY h ORDER BY h; -``` - -Bazı durumlarda yandex'teki arkadaş ekiplerimizin test ortamına da dağıtım yapıyoruz: Pazar, Bulut, vb. Ayrıca geliştirme amacıyla kullanılan bazı donanım sunucularımız var. - -## Yük Testi {#load-testing} - -Test ortamına dağıtıldıktan sonra, üretim kümesinden gelen sorgularla yük testini çalıştırıyoruz. Bu elle yapılır. - -Etkinleştirdiğinizden emin olun `query_log` üretim kümenizde. - -Bir gün veya daha fazla sorgu günlüğü toplayın: - -``` bash -$ clickhouse-client --query="SELECT DISTINCT query FROM system.query_log WHERE event_date = today() AND query LIKE '%ym:%' AND query NOT LIKE '%system.query_log%' AND type = 2 AND is_initial_query" > queries.tsv -``` - -Bu şekilde karmaşık bir örnektir. `type = 2` başarıyla yürütülen sorguları süzer. `query LIKE '%ym:%'` yandex'ten ilgili sorguları seçmektir.Metrica. `is_initial_query` yalnızca istemci tarafından başlatılan sorguları seçmektir, Clickhouse'un kendisi tarafından değil (dağıtılmış sorgu işlemenin parçaları olarak). - -`scp` bu test kümenize günlük ve aşağıdaki gibi çalıştırın: - -``` bash -$ clickhouse benchmark --concurrency 16 < queries.tsv -``` - -(muhtemelen de belirtmek istiyorum `--user`) - -Sonra bir gece ya da hafta sonu için bırakın ve dinlenin. - -Kontrol etmelisiniz `clickhouse-server` çökmez, bellek ayak izi sınırlıdır ve performans zamanla aşağılayıcı değildir. - -Kesin sorgu yürütme zamanlamaları kaydedilmez ve sorguların ve ortamın yüksek değişkenliği nedeniyle karşılaştırılmaz. - -## Yapı Testleri {#build-tests} - -Yapı testleri, yapının çeşitli alternatif konfigürasyonlarda ve bazı yabancı sistemlerde bozulmadığını kontrol etmeyi sağlar. Testler bulunur `ci` dizin. Docker, Vagrant ve bazen de `qemu-user-static` Docker'ın içinde. Bu testler geliştirme aşamasındadır ve test çalıştırmaları otomatik değildir. - -Motivasyon: - -Normalde tüm testleri ClickHouse yapısının tek bir varyantında serbest bırakırız ve çalıştırırız. Ancak, iyice test edilmeyen alternatif yapı varyantları vardır. Örnekler: - -- FreeBSD üzerine inşa; -- sistem paketlerinden kütüphaneler ile Debian üzerine inşa; -- kütüphanelerin paylaşılan bağlantısı ile oluşturun; -- AArch64 platformunda oluşturun; -- PowerPc platformunda oluşturun. - -Örneğin, sistem paketleri ile oluştur kötü bir uygulamadır, çünkü bir sistemin hangi paketlerin tam sürümüne sahip olacağını garanti edemeyiz. Ancak bu gerçekten Debian bakıcılarına ihtiyaç duyuyor. Bu nedenle en azından bu yapı varyantını desteklemeliyiz. Başka bir örnek: paylaşılan bağlantı ortak bir sorun kaynağıdır, ancak bazı Meraklılar için gereklidir. - -Tüm yapı varyantlarında tüm testleri çalıştıramasak da, en azından çeşitli yapı varyantlarının bozulmadığını kontrol etmek istiyoruz. Bu amaçla yapı testlerini kullanıyoruz. - -## Protokol uyumluluğu testi {#testing-for-protocol-compatibility} - -ClickHouse ağ protokolünü genişlettiğimizde, eski clickhouse istemcisinin yeni clickhouse sunucusu ile çalıştığını ve yeni clickhouse istemcisinin eski clickhouse sunucusu ile çalıştığını (sadece ilgili paketlerden ikili dosyaları çalıştırarak) manuel olarak test ediyoruz. - -## Derleyiciden yardım {#help-from-the-compiler} - -Ana ClickHouse kodu (bu `dbms` dizin) ile inşa edilmiştir `-Wall -Wextra -Werror` ve bazı ek etkin uyarılar ile. Bu seçenekler üçüncü taraf kitaplıkları için etkin olmasa da. - -Clang daha yararlı uyarılar vardır-Sen ile onları arayabilirsiniz `-Weverything` ve varsayılan oluşturmak için bir şey seçin. - -Üretim yapıları için gcc kullanılır (hala clang'dan biraz daha verimli kod üretir). Geliştirme için, clang genellikle kullanımı daha uygundur. Hata ayıklama modu ile kendi makinenizde inşa edebilirsiniz (dizüstü bilgisayarınızın pilinden tasarruf etmek için), ancak derleyicinin daha fazla uyarı üretebileceğini lütfen unutmayın `-O3` daha iyi kontrol akışı ve prosedürler arası analiz nedeniyle. Clang ile inşa ederken ayıklama modu ile oluştururken, hata ayıklama sürümü `libc++` çalışma zamanında daha fazla hata yakalamak için izin verir kullanılır. - -## Dezenfektanlar {#sanitizers} - -**Adres dezenfektanı**. -Biz başına taahhüt bazında ASan altında fonksiyonel ve entegrasyon testleri çalıştırın. - -**Valgrind (Memcheck)**. -Bir gecede valgrind altında fonksiyonel testler yapıyoruz. Birden fazla saat sürer. Şu anda bilinen bir yanlış pozitif var `re2` kütüphane, bkz [bu makale](https://research.swtch.com/sparse). - -**Tanımsız davranış dezenfektanı.** -Biz başına taahhüt bazında ASan altında fonksiyonel ve entegrasyon testleri çalıştırın. - -**İplik dezenfektanı**. -Biz başına taahhüt bazında tsan altında fonksiyonel testler çalıştırın. Tsan altında hala taahhüt bazında entegrasyon testleri yapmıyoruz. - -**Bellek temizleyici**. -Şu anda hala MSan kullanmıyoruz. - -**Hata ayıklama ayırıcısı.** -Hata ayıklama sürümü `jemalloc` hata ayıklama oluşturmak için kullanılır. - -## Fuzzing {#fuzzing} - -ClickHouse fuzzing hem kullanılarak uygulanmaktadır [libFuzzer](https://llvm.org/docs/LibFuzzer.html) ve rastgele SQL sorguları. -Tüm fuzz testleri sanitizers (Adres ve tanımsız) ile yapılmalıdır. - -LibFuzzer kütüphane kodu izole fuzz testi için kullanılır. Fuzzers test kodunun bir parçası olarak uygulanır ve “_fuzzer” adı postfixes. -Fuzzer örneği bulunabilir `src/Parsers/tests/lexer_fuzzer.cpp`. LibFuzzer özgü yapılandırmalar, sözlükler ve corpus saklanır `tests/fuzz`. -Kullanıcı girişini işleyen her işlevsellik için fuzz testleri yazmanızı öneririz. - -Fuzzers varsayılan olarak oluşturulmaz. Hem fuzzers inşa etmek `-DENABLE_FUZZING=1` ve `-DENABLE_TESTS=1` seçenekler ayarlanmalıdır. -Fuzzers oluştururken Jemalloc'u devre dışı bırakmanızı öneririz. ClickHouse fuzzing'i entegre etmek için kullanılan yapılandırma -Google OSS-Fuzz bulunabilir `docker/fuzz`. - -Ayrıca rastgele SQL sorguları oluşturmak ve sunucunun bunları çalıştırarak ölmediğini kontrol etmek için basit fuzz testi kullanıyoruz. -İçinde bulabilirsiniz `00746_sql_fuzzy.pl`. Bu test sürekli olarak (gece ve daha uzun) çalıştırılmalıdır. - -## Güvenlik Denetimi {#security-audit} - -Yandex Güvenlik ekibinden insanlar güvenlik açısından ClickHouse yetenekleri bazı temel bakış yapmak. - -## Statik Analizörler {#static-analyzers} - -Koş weuyoruz `PVS-Studio` taahhüt bazında. Değerlendir havedik `clang-tidy`, `Coverity`, `cppcheck`, `PVS-Studio`, `tscancode`. Sen kullanım talimatları bulacaksınız `tests/instructions/` dizin. Ayrıca okuyabilirsiniz [Rusça makale](https://habr.com/company/yandex/blog/342018/). - -Kullanıyorsanız `CLion` bir IDE olarak, bazı kaldıraç `clang-tidy` kutudan kontrol eder. - -## Sertleşme {#hardening} - -`FORTIFY_SOURCE` varsayılan olarak kullanılır. Neredeyse işe yaramaz, ancak nadir durumlarda hala mantıklı ve bunu devre dışı bırakmıyoruz. - -## Kod Stili {#code-style} - -Kod stili kuralları açıklanmıştır [burada](https://clickhouse.tech/docs/en/development/style/). - -Bazı ortak stil ihlallerini kontrol etmek için şunları kullanabilirsiniz `utils/check-style` komut. - -Kodunuzun uygun stilini zorlamak için şunları kullanabilirsiniz `clang-format`. Dosya `.clang-format` kaynak rootlarında yer almaktadır. Çoğunlukla gerçek kod stilimizle karşılık gelir. Ancak uygulanması tavsiye edilmez `clang-format` varolan dosyalara biçimlendirmeyi daha da kötüleştirdiği için. Kullanabilirsiniz `clang-format-diff` eğer clang kaynak deposunda bulabilirsiniz aracı. - -Alternatif olarak deneyebilirsiniz `uncrustify` kodunuzu yeniden biçimlendirmek için bir araç. Yapılandırma içinde `uncrustify.cfg` kaynaklarda kök. Daha az test edilmiştir `clang-format`. - -`CLion` kod stilimiz için ayarlanması gereken kendi kod biçimlendiricisine sahiptir. - -## Metrica B2B testleri {#metrica-b2b-tests} - -Her ClickHouse sürümü Yandex Metrica ve AppMetrica motorları ile test edilir. Clickhouse'un Test ve kararlı sürümleri Vm'lerde dağıtılır ve Giriş verilerinin sabit örneğini işleyen Metrica motorunun küçük bir kopyasıyla çalışır. Daha sonra Metrica motorunun iki örneğinin sonuçları birlikte karşılaştırılır. - -Bu testler ayrı ekip tarafından otomatikleştirilir. Yüksek sayıda hareketli parça nedeniyle, testler çoğu zaman tamamen ilgisiz nedenlerle başarısız olur, bu da anlaşılması çok zordur. Büyük olasılıkla bu testlerin bizim için negatif değeri var. Bununla birlikte, bu testlerin yüzlerce kişiden yaklaşık bir veya iki kez yararlı olduğu kanıtlanmıştır. - -## Test Kapsamı {#test-coverage} - -Temmuz 2018 itibariyle test kapsamını takip etmiyoruz. - -## Test Otomasyonu {#test-automation} - -Yandex dahili CI ve iş otomasyon sistemi ile testler yapıyoruz “Sandbox”. - -Yapı işleri ve testler, taahhüt bazında sanal alanda çalıştırılır. Ortaya çıkan paketler ve test sonuçları Github'da yayınlanır ve doğrudan bağlantılar tarafından indirilebilir. Eserler sonsuza dek saklanır. Eğer GitHub bir çekme isteği gönderdiğinizde, biz olarak etiketlemek “can be tested” ve bizim CI sistemi sizin için ClickHouse paketleri (yayın, hata ayıklama, Adres dezenfektanı ile, vb) inşa edecek. - -Travis CI, zaman ve hesaplama gücü sınırı nedeniyle kullanmıyoruz. -Jenkins'i kullanmayız. Daha önce kullanıldı ve şimdi Jenkins kullanmadığımız için mutluyuz. - -[Orijinal makale](https://clickhouse.tech/docs/en/development/tests/) diff --git a/docs/tr/development/tests.md b/docs/tr/development/tests.md new file mode 120000 index 00000000000..c03d36c3916 --- /dev/null +++ b/docs/tr/development/tests.md @@ -0,0 +1 @@ +../../en/development/tests.md \ No newline at end of file diff --git a/docs/zh/development/tests.md b/docs/zh/development/tests.md deleted file mode 100644 index d2690a86887..00000000000 --- a/docs/zh/development/tests.md +++ /dev/null @@ -1,237 +0,0 @@ -# ClickHouse 测试 {#clickhouse-ce-shi} - -## 功能性测试 {#gong-neng-xing-ce-shi} - -功能性测试是最简便使用的。绝大部分 ClickHouse 的功能可以通过功能性测试来测试,任何代码的更改都必须通过该测试。 - -每个功能测试会向正在运行的 ClickHouse服务器发送一个或多个查询,并将结果与预期结果进行比较。 - -测试用例在 `tests/queries` 目录中。这里有两个子目录:`stateless` 和 `stateful`目录。无状态的测试无需预加载测试数据集 - 通常是在测试运行期间动态创建小量的数据集。有状态测试需要来自 Yandex.Metrica 的预加载测试数据,而不向一般公众提供。我们倾向于仅使用«无状态»测试并避免添加新的«有状态»测试。 - -每个测试用例可以是两种类型之一:`.sql` 和 `.sh`。`.sql` 测试文件是用于管理`clickhouse-client --multiquery --testmode`的简单SQL脚本。`.sh` 测试文件是一个可以自己运行的脚本。 - -要运行所有测试,请使用 `tests/clickhouse-test` 工具,用 `--help` 可以获取所有的选项列表。您可以简单地运行所有测试或运行测试名称中的子字符串过滤的测试子集:`./clickhouse-test substring`。 - -调用功能测试最简单的方法是将 `clickhouse-client` 复制到`/usr/bin/`,运行`clickhouse-server`,然后从自己的目录运行`./ clickhouse-test`。 - -要添加新测试,请在 `tests/queries/0_stateless` 目录内添加新的 `.sql` 或 `.sh` 文件,手动检查,然后按以下方式生成 `.reference` 文件: `clickhouse-client -n --testmode < 00000_test.sql > 00000_test.reference` 或 `./00000_test.sh > ./00000_test.reference`。 - -测试应该只使用(创建,删除等)`test` 数据库中的表,这些表假定是事先创建的; 测试也可以使用临时表。 - -如果要在功能测试中使用分布式查询,可以利用 `remote` 表函数和 `127.0.0.{1..2}` 地址为服务器查询自身; 或者您可以在服务器配置文件中使用预定义的测试集群,例如`test_shard_localhost`。 - -有些测试在名称中标有 `zookeeper`,`shard` 或 `long`。`zookeeper` 用于使用ZooKeeper的测试; `shard` 用于需要服务器监听`127.0.0.*`的测试。`long` 适用于运行时间稍长一秒的测试。 - -## 已知的bug {#yi-zhi-de-bug} - -如果我们知道一些可以通过功能测试轻松复制的错误,我们将准备好的功能测试放在 `tests/queries/bugs` 目录中。当修复错误时,这些测试将被移动到 `tests/queries/0_stateless` 目录中。 - -## 集成测试 {#ji-cheng-ce-shi} - -集成测试允许在集群配置中测试 ClickHouse,并与其他服务器(如MySQL,Postgres,MongoDB)进行 ClickHouse 交互。它们可用于模拟网络拆分,数据包丢弃等。这些测试在Docker下运行,并使用各种软件创建多个容器。 - -参考 `tests/integration/README.md` 文档关于如何使用集成测试。 - -请注意,ClickHouse 与第三方驱动程序的集成未经过测试。此外,我们目前还没有与 JDBC 和ODBC 驱动程序进行集成测试。 - -## 单元测试 {#dan-yuan-ce-shi} - -当您想要测试整个 ClickHouse,而不是单个独立的库或类时,单元测试非常有用。您可以使用`ENABLE_TESTS` CMake 选项启用或禁用测试构建。单元测试(和其他测试程序)位于代码中的`tests` 子目录中。要运行单元测试,请键入 `ninja test`。有些测试使用 `gtest`,但有些只是在测试失败时返回非零状态码。 - -如果代码已经被功能测试覆盖(并且功能测试通常使用起来要简单得多),则不一定要进行单元测试。 - -## 性能测试 {#xing-neng-ce-shi} - -性能测试允许测量和比较综合查询中 ClickHouse 的某些独立部分的性能。测试位于`tests/performance` 目录中。每个测试都由 `.xml` 文件表示,并附有测试用例的描述。使用 `clickhouse performance-test` 工具(嵌入在 `clickhouse` 二进制文件中)运行测试。请参阅 `--help` 以进行调用。 - -每个测试在循环中运行一个或多个查询(可能带有参数组合),并具有一些停止条件(如«最大执行速度不会在三秒内更改»)并测量一些有关查询性能的指标(如«最大执行速度»))。某些测试可以包含预加载的测试数据集的前提条件。 - -如果要在某些情况下提高 ClickHouse 的性能,并且如果可以在简单查询上观察到改进,则强烈建议编写性能测试。在测试过程中使用 `perf top` 或其他 perf 工具总是有意义的。 - -性能测试不是基于每个提交运行的。不收集性能测试结果,我们手动比较它们。 - -## 测试工具和脚本 {#ce-shi-gong-ju-he-jiao-ben} - -`tests`目录中的一些程序不是准备测试,而是测试工具。例如,对于`Lexer`,有一个工具`src/Parsers/tests/lexer` 标准输出。您可以使用这些工具作为代码示例以及探索和手动测试。 - -您还可以将一对文件 `.sh` 和 `.reference` 与工具放在一些预定义的输入上运行它 - 然后可以将脚本结果与 `.reference` 文件进行比较。这些测试不是自动化的。 - -## 杂项测试 {#za-xiang-ce-shi} - -有一些外部字典的测试位于 `tests/external_dictionaries`,机器学习模型在`tests/external_models`目录。这些测试未更新,必须转移到集成测试。 - -对于分布式数据的插入,有单独的测试。此测试在单独的服务器上运行 ClickHouse 集群并模拟各种故障情况:网络拆分,数据包丢弃(ClickHouse 节点之间,ClickHouse 和 ZooKeeper之间,ClickHouse 服务器和客户端之间等),进行 `kill -9`,`kill -STOP` 和`kill -CONT` 等操作,类似[Jepsen](https://aphyr.com/tags/Jepsen)。然后,测试检查是否已写入所有已确认的插入,并且所有已拒绝的插入都未写入。 - -在 ClickHouse 开源之前,分布式测试是由单独的团队编写的,但该团队不再使用 ClickHouse,测试是在 Java 中意外编写的。由于这些原因,必须重写分布式测试并将其移至集成测试。 - -## 手动测试 {#shou-dong-ce-shi} - -当您开发了新的功能,做手动测试也是合理的。可以按照以下步骤来进行: - -编译 ClickHouse。在命令行中运行 ClickHouse:进入 `programs/clickhouse-server` 目录并运行 `./clickhouse-server`。它会默认使用当前目录的配置文件 (`config.xml`, `users.xml` 以及在 `config.d` 和 `users.d` 目录的文件)。可以使用 `programs/clickhouse-client/clickhouse-client` 来连接数据库。 - -或者,您可以安装 ClickHouse 软件包:从 Yandex 存储库中获得稳定版本,或者您可以在ClickHouse源根目录中使用 `./release` 构建自己的软件包。然后使用 `sudo service clickhouse-server start` 启动服务器(或停止服务器)。在 `/etc/clickhouse-server/clickhouse-server.log` 中查找日志。 - -当您的系统上已经安装了 ClickHouse 时,您可以构建一个新的 `clickhouse` 二进制文件并替换现有的二进制文件: - - sudo service clickhouse-server stop - sudo cp ./clickhouse /usr/bin/ - sudo service clickhouse-server start - -您也可以停止 clickhouse-server 并使用相同的配置运行您自己的服务器,日志打印到终端: - - sudo service clickhouse-server stop - sudo -u clickhouse /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml - -使用 gdb 的一个示例: - - sudo -u clickhouse gdb --args /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml - -如果 clickhouse-server 已经运行并且您不想停止它,您可以更改 `config.xml` 中的端口号(或在 `config.d` 目录中的文件中覆盖它们),配置适当的数据路径,然后运行它。 - -`clickhouse` 二进制文件几乎没有依赖关系,适用于各种 Linux 发行版。要快速地测试服务器上的更改,您可以简单地将新建的 `clickhouse` 二进制文件 `scp` 到其他服务器,然后按照上面的示例运行它。 - -## 测试环境 {#ce-shi-huan-jing} - -在将版本发布为稳定之前,我们将其部署在测试环境中测试环境是一个处理\[Yandex.Metrica\](https://metrica.yandex.com/)总数据的1/39部分大小的集群。我们与 Yandex.Metrica 团队公用我们的测试环境。ClickHouse 在现有数据的基础上无需停机即可升级。我们首先看到数据处理成功而不会实时滞后,复制继续工作,并且 Yandex.Metrica 团队无法看到问题。首先的检查可以通过以下方式完成: - - SELECT hostName() AS h, any(version()), any(uptime()), max(UTCEventTime), count() FROM remote('example01-01-{1..3}t', merge, hits) WHERE EventDate >= today() - 2 GROUP BY h ORDER BY h; - -在某些情况下,我们还部署到 Yandex 的合作团队的测试环境:市场,云等。此外,我们还有一些用于开发目的的硬件服务器。 - -## 负载测试 {#fu-zai-ce-shi} - -部署到测试环境后,我们使用生产群集中的查询运行负载测试。这是手动完成的。 - -确保在生产集群中开启了 `query_log` 选项。 - -收集一天或更多的查询日志: - - clickhouse-client --query="SELECT DISTINCT query FROM system.query_log WHERE event_date = today() AND query LIKE '%ym:%' AND query NOT LIKE '%system.query_log%' AND type = 2 AND is_initial_query" > queries.tsv - -这是一个复杂的例子。`type = 2` 将过滤成功执行的查询。`query LIKE'%ym:%'` 用于从 Yandex.Metrica 中选择相关查询。`is_initial_query` 是仅选择由客户端发起的查询,而不是由 ClickHouse 本身(作为分布式查询处理的一部分)。 - -`scp` 这份日志到测试机器,并运行以下操作: - - clickhouse benchmark --concurrency 16 < queries.tsv - -(可能你需要指定运行的用户 `--user`) - -然后离开它一晚或周末休息一下。 - -你要检查下 `clickhouse-server` 是否崩溃,内存占用是否合理,性能也不会随着时间的推移而降低。 - -由于查询和环境的高度可变性,不会记录精确的查询执行时序并且不进行比较。 - -## 编译测试 {#bian-yi-ce-shi} - -构建测试允许检查构建在各种替代配置和某些外部系统上是否被破坏。测试位于`ci`目录。它们从 Docker,Vagrant 中的源代码运行构建,有时在 Docker 中运行 `qemu-user-static`。这些测试正在开发中,测试运行不是自动化的。 - -动机: - -通常我们会在 ClickHouse 构建的单个版本上发布并运行所有测试。但是有一些未经过彻底测试的替代构建版本。例子: - -- 在 FreeBSD 中的构建; -- 在 Debian 中使用系统包中的库进行构建; -- 使用库的共享链接构建; -- 在 AArch64 平台进行构建。 - -例如,使用系统包构建是不好的做法,因为我们无法保证系统具有的确切版本的软件包。但 Debian 维护者确实需要这样做。出于这个原因,我们至少必须支持这种构建。另一个例子:共享链接是一个常见的麻烦来源,但是对于一些爱好者来说需要它。 - -虽然我们无法对所有构建版本运行所有测试,但我们想要检查至少不会破坏各种构建变体。为此,我们使用构建测试。 - -## 测试协议兼容性 {#ce-shi-xie-yi-jian-rong-xing} - -当我们扩展 ClickHouse 网络协议时,我们手动测试旧的 clickhouse-client 与新的 clickhouse-server 和新的clickhouse-client 一起使用旧的 clickhouse-server (只需从相应的包中运行二进制文件) - -## 来自编译器的提示 {#lai-zi-bian-yi-qi-de-ti-shi} - -ClickHouse 主要的代码 (位于`dbms`目录中) 使用 `-Wall -Wextra -Werror` 构建,并带有一些其他已启用的警告。 虽然没有为第三方库启用这些选项。 - -Clang 有更多有用的警告 - 您可以使用 `-Weverything` 查找它们并选择默认构建的东西。 - -对于生产构建,使用 gcc(它仍然生成比 clang 稍高效的代码)。对于开发来说,clang 通常更方便使用。您可以使用调试模式在自己的机器上构建(以节省笔记本电脑的电量),但请注意,由于更好的控制流程和过程分析,编译器使用 `-O3` 会生成更多警告。 当使用 clang 构建时,使用 `libc++` 而不是 `libstdc++`,并且在使用调试模式构建时,使用调试版本的 `libc++`,它允许在运行时捕获更多错误。 - -## Sanitizers {#sanitizers} - -### Address sanitizer -我们使用Asan对每个提交进行功能和集成测试。 - -### Valgrind (Memcheck) -我们在夜间使用Valgrind进行功能测试。这需要几个小时。目前在 `re2` 库中有一个已知的误报,请参阅[文章](https://research.swtch.com/sparse)。 - -### Undefined behaviour sanitizer -我们使用Asan对每个提交进行功能和集成测试。 - -### Thread sanitizer -我们使用TSan对每个提交进行功能测试。目前不使用TSan对每个提交进行集成测试。 - -### Memory sanitizer -目前我们不使用 MSan。 - -### Debug allocator -您可以使用 `DEBUG_TCMALLOC` CMake 选项启用 `tcmalloc` 的调试版本。我们在每次提交的基础上使用调试分配器运行测试。 - -更多请参阅 `tests/instructions/sanitizers.txt`。 - -## 模糊测试 {#mo-hu-ce-shi} - -ClickHouse模糊测试可以通过[libFuzzer](https://llvm.org/docs/LibFuzzer.html)和随机SQL查询实现。 -所有的模糊测试都应使用sanitizers(Address及Undefined)。 - -LibFuzzer用于对库代码进行独立的模糊测试。模糊器作为测试代码的一部分实现,并具有“_fuzzer”名称后缀。 -模糊测试示例在`src/Parsers/tests/lexer_fuzzer.cpp`。LibFuzzer配置、字典及语料库存放在`tests/fuzz`。 -我们鼓励您为每个处理用户输入的功能编写模糊测试。 - -默认情况下不构建模糊器。可通过设置`-DENABLE_FUZZING=1`和`-DENABLE_TESTS=1`来构建模糊器。 我们建议在构建模糊器时关闭Jemalloc。 -用于将ClickHouse模糊测试集成到的Google OSS-Fuzz的配置文件位于`docker/fuzz`。 - -此外,我们使用简单的模糊测试来生成随机SQL查询并检查服务器是否正常。你可以在`00746_sql_fuzzy.pl` 找到它。测试应连续进行(过夜和更长时间)。 - -## 安全审计 {#an-quan-shen-ji} - -Yandex Cloud 部门的人员从安全角度对 ClickHouse 功能进行了一些基本概述。 - -## 静态分析 {#jing-tai-fen-xi} - -我们偶尔使用静态分析。我们已经评估过 `clang-tidy`, `Coverity`, `cppcheck`, `PVS-Studio`, `tscancode`。您将在 `tests/instructions/` 目录中找到使用说明。你也可以阅读[俄文文章](https://habr.com/company/yandex/blog/342018/). - -如果您使用 `CLion` 作为 IDE,您可以开箱即用一些 `clang-tidy` 检查。 - -## 其他强化 {#qi-ta-qiang-hua} - -默认情况下使用 `FORTIFY_SOURCE`。它几乎没用,但在极少数情况下仍然有意义,我们不会禁用它。 - -## 代码风格 {#dai-ma-feng-ge} - -代码风格在[这里](https://clickhouse.tech/docs/en/development/style/) 有说明。 - -要检查一些常见的样式冲突,您可以使用 `utils/check-style` 脚本。 - -为了强制你的代码的正确风格,你可以使用 `clang-format` 文件。`.clang-format` 位于源代码根目录, 它主要与我们的实际代码风格对应。但不建议将 `clang-format` 应用于现有文件,因为它会使格式变得更糟。您可以使用 `clang-format-diff` 工具,您可以在 clang 源代码库中找到 - -或者,您可以尝试`uncrustify` 工具来格式化您的代码。配置文件在源代码的根目录中的`uncrustify.cfg`。它比 `clang-format` 经过更少的测试。 - -`CLion` 有自己的代码格式化程序,必须调整为我们的代码风格。 - -## Metrica B2B 测试 {#metrica-b2b-ce-shi} - -每个 ClickHouse 版本都经过 Yandex Metrica 和 AppMetrica 引擎的测试。测试和稳定版本的 ClickHouse 部署在虚拟机上,并使用处理输入数据固定样本的度量引擎的小副本运行。将度量引擎的两个实例的结果一起进行比较 - -这些测试是由单独的团队自动完成的。由于移动部件的数量很多,大部分时间的测试都是完全无关的,很难弄清楚。很可能这些测试对我们来说是负值。然而,这些测试被证明是有用的大约一个或两个倍的数百。 - -## 测试覆盖率 {#ce-shi-fu-gai-lu} - -截至2018年7月,我们不会跟踪测试复盖率。 - -## 自动化测试 {#zi-dong-hua-ce-shi} - -我们使用 Yandex 内部 CI 和名为«沙箱»的作业自动化系统运行测试。我们还继续使用 Jenkins(可在Yandex内部使用)。 - -构建作业和测试在沙箱中按每次提交的基础上运行。结果包和测试结果发布在 GitHub 上,可以通过直接链接下载,结果会被永久存储。当您在 GitHub 上发送拉取请求时,我们将其标记为«可以测试»,我们的 CI 系统将为您构建 ClickHouse 包(发布,调试,地址消除等)。 - -由于时间和计算能力的限制,我们不使用 Travis CI。 - -在 Jenkins,我们运行字典测试,指标B2B测试。我们使用 Jenkins 来准备和发布版本。Jenkins是一种传统的技术,所有的工作将被转移到沙箱中。 - -[来源文章](https://clickhouse.tech/docs/zh/development/tests/) From 4c6819c26fa8c608e9f4d004a408bcaf680467d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 17:41:16 +0300 Subject: [PATCH 198/314] Fix misleading info --- docs/en/development/tests.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index e6141f7ab02..5b096bcc5fa 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -74,9 +74,9 @@ It’s not necessarily to have unit tests if the code is already covered by func ## Performance Tests {#performance-tests} -Performance tests allow to measure and compare performance of some isolated part of ClickHouse on synthetic queries. Tests are located at `tests/performance`. Each test is represented by `.xml` file with description of test case. Tests are run with `clickhouse performance-test` tool (that is embedded in `clickhouse` binary). See `--help` for invocation. +Performance tests allow to measure and compare performance of some isolated part of ClickHouse on synthetic queries. Tests are located at `tests/performance`. Each test is represented by `.xml` file with description of test case. Tests are run with `docker/tests/performance-comparison` tool . See the readme file for invocation. -Each test run one or multiple queries (possibly with combinations of parameters) in a loop with some conditions for stop (like “maximum execution speed is not changing in three seconds”) and measure some metrics about query performance (like “maximum execution speed”). Some tests can contain preconditions on preloaded test dataset. +Each test run one or multiple queries (possibly with combinations of parameters) in a loop. Some tests can contain preconditions on preloaded test dataset. If you want to improve performance of ClickHouse in some scenario, and if improvements can be observed on simple queries, it is highly recommended to write a performance test. It always makes sense to use `perf top` or other perf tools during your tests. From d966bb939efc0c7d65e56ce816b5ca5546b4a1e1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 17:51:41 +0300 Subject: [PATCH 199/314] Fix inconsistent code around H3 --- src/Functions/CMakeLists.txt | 18 ------------------ src/Functions/geoToH3.cpp | 8 ++++++++ src/Functions/h3EdgeAngle.cpp | 8 ++++++++ src/Functions/h3EdgeLengthM.cpp | 8 ++++++++ src/Functions/h3GetBaseCell.cpp | 8 ++++++++ src/Functions/h3GetResolution.cpp | 8 ++++++++ src/Functions/h3HexAreaM2.cpp | 8 ++++++++ src/Functions/h3IndexesAreNeighbors.cpp | 8 ++++++++ src/Functions/h3IsValid.cpp | 8 ++++++++ src/Functions/h3ToChildren.cpp | 8 ++++++++ src/Functions/h3ToParent.cpp | 8 ++++++++ src/Functions/h3ToString.cpp | 8 ++++++++ src/Functions/h3kRing.cpp | 8 ++++++++ src/Functions/stringToH3.cpp | 8 ++++++++ 14 files changed, 104 insertions(+), 18 deletions(-) diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index bdf89c983f1..126e4c7c57d 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -6,24 +6,6 @@ add_headers_and_sources(clickhouse_functions .) list(REMOVE_ITEM clickhouse_functions_sources IFunctionImpl.cpp FunctionFactory.cpp FunctionHelpers.cpp) list(REMOVE_ITEM clickhouse_functions_headers IFunctionImpl.h FunctionFactory.h FunctionHelpers.h) -if (NOT USE_H3) - list (REMOVE_ITEM clickhouse_functions_sources - geoToH3.cpp - h3EdgeAngle.cpp - h3EdgeLengthM.cpp - h3GetBaseCell.cpp - h3GetResolution.cpp - h3HexAreaM2.cpp - h3IndexesAreNeighbors.cpp - h3IsValid.cpp - h3kRing.cpp - h3ToChildren.cpp - h3ToParent.cpp - h3ToString.cpp - stringToH3.cpp - ) -endif () - add_library(clickhouse_functions ${clickhouse_functions_sources}) target_link_libraries(clickhouse_functions diff --git a/src/Functions/geoToH3.cpp b/src/Functions/geoToH3.cpp index 882425c4a77..257c73daa4c 100644 --- a/src/Functions/geoToH3.cpp +++ b/src/Functions/geoToH3.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + +#if USE_H3 + #include #include #include @@ -94,3 +100,5 @@ void registerFunctionGeoToH3(FunctionFactory & factory) } } + +#endif diff --git a/src/Functions/h3EdgeAngle.cpp b/src/Functions/h3EdgeAngle.cpp index 88995427e6d..7b0b1cdc3a7 100644 --- a/src/Functions/h3EdgeAngle.cpp +++ b/src/Functions/h3EdgeAngle.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + +#if USE_H3 + #include #include #include @@ -77,3 +83,5 @@ void registerFunctionH3EdgeAngle(FunctionFactory & factory) } } + +#endif diff --git a/src/Functions/h3EdgeLengthM.cpp b/src/Functions/h3EdgeLengthM.cpp index 6626deffd6b..d22074ceee0 100644 --- a/src/Functions/h3EdgeLengthM.cpp +++ b/src/Functions/h3EdgeLengthM.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + +#if USE_H3 + #include #include #include @@ -81,3 +87,5 @@ void registerFunctionH3EdgeLengthM(FunctionFactory & factory) } } + +#endif diff --git a/src/Functions/h3GetBaseCell.cpp b/src/Functions/h3GetBaseCell.cpp index 80cbf96fe9b..f9bc8bcea20 100644 --- a/src/Functions/h3GetBaseCell.cpp +++ b/src/Functions/h3GetBaseCell.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + +#if USE_H3 + #include #include #include @@ -70,3 +76,5 @@ void registerFunctionH3GetBaseCell(FunctionFactory & factory) } } + +#endif diff --git a/src/Functions/h3GetResolution.cpp b/src/Functions/h3GetResolution.cpp index 1b7d108dbd1..c73d7ed4f4c 100644 --- a/src/Functions/h3GetResolution.cpp +++ b/src/Functions/h3GetResolution.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + +#if USE_H3 + #include #include #include @@ -70,3 +76,5 @@ void registerFunctionH3GetResolution(FunctionFactory & factory) } } + +#endif diff --git a/src/Functions/h3HexAreaM2.cpp b/src/Functions/h3HexAreaM2.cpp index 8de17cc1f11..f1e74591785 100644 --- a/src/Functions/h3HexAreaM2.cpp +++ b/src/Functions/h3HexAreaM2.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + +#if USE_H3 + #include #include #include @@ -76,3 +82,5 @@ void registerFunctionH3HexAreaM2(FunctionFactory & factory) } } + +#endif diff --git a/src/Functions/h3IndexesAreNeighbors.cpp b/src/Functions/h3IndexesAreNeighbors.cpp index d2ba36347a3..bddaffd96bc 100644 --- a/src/Functions/h3IndexesAreNeighbors.cpp +++ b/src/Functions/h3IndexesAreNeighbors.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + +#if USE_H3 + #include #include #include @@ -78,3 +84,5 @@ void registerFunctionH3IndexesAreNeighbors(FunctionFactory & factory) } } + +#endif diff --git a/src/Functions/h3IsValid.cpp b/src/Functions/h3IsValid.cpp index 9455bec19ee..bd99f57af3e 100644 --- a/src/Functions/h3IsValid.cpp +++ b/src/Functions/h3IsValid.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + +#if USE_H3 + #include #include #include @@ -70,3 +76,5 @@ void registerFunctionH3IsValid(FunctionFactory & factory) } } + +#endif diff --git a/src/Functions/h3ToChildren.cpp b/src/Functions/h3ToChildren.cpp index d1438c70676..8c0d4d23e96 100644 --- a/src/Functions/h3ToChildren.cpp +++ b/src/Functions/h3ToChildren.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + +#if USE_H3 + #include #include #include @@ -111,3 +117,5 @@ void registerFunctionH3ToChildren(FunctionFactory & factory) } } + +#endif diff --git a/src/Functions/h3ToParent.cpp b/src/Functions/h3ToParent.cpp index 1f3cafe690f..cbdbbedfcd6 100644 --- a/src/Functions/h3ToParent.cpp +++ b/src/Functions/h3ToParent.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + +#if USE_H3 + #include #include #include @@ -85,3 +91,5 @@ void registerFunctionH3ToParent(FunctionFactory & factory) } } + +#endif diff --git a/src/Functions/h3ToString.cpp b/src/Functions/h3ToString.cpp index 5f1a93648ad..01d1b02809a 100644 --- a/src/Functions/h3ToString.cpp +++ b/src/Functions/h3ToString.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + +#if USE_H3 + #include #include #include @@ -86,3 +92,5 @@ void registerFunctionH3ToString(FunctionFactory & factory) } } + +#endif diff --git a/src/Functions/h3kRing.cpp b/src/Functions/h3kRing.cpp index 10fea799dd2..883c20d6324 100644 --- a/src/Functions/h3kRing.cpp +++ b/src/Functions/h3kRing.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + +#if USE_H3 + #include #include #include @@ -108,3 +114,5 @@ void registerFunctionH3KRing(FunctionFactory & factory) } } + +#endif diff --git a/src/Functions/stringToH3.cpp b/src/Functions/stringToH3.cpp index 10cb3120c3f..65e7fa20314 100644 --- a/src/Functions/stringToH3.cpp +++ b/src/Functions/stringToH3.cpp @@ -1,3 +1,9 @@ +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif + +#if USE_H3 + #include #include #include @@ -101,3 +107,5 @@ void registerFunctionStringToH3(FunctionFactory & factory) } } + +#endif From b8b9c6d40997416b5f5fabdaf7af27d53e07b6cc Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 4 Nov 2020 17:55:05 +0300 Subject: [PATCH 200/314] Update reinterpretAs.cpp --- src/Functions/reinterpretAs.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Functions/reinterpretAs.cpp b/src/Functions/reinterpretAs.cpp index 76ee6155048..1c309505ac9 100644 --- a/src/Functions/reinterpretAs.cpp +++ b/src/Functions/reinterpretAs.cpp @@ -40,8 +40,9 @@ class FunctionReinterpretAs : public IFunction template static void reinterpretImpl(const PaddedPODArray & from, PaddedPODArray & to) { - to.resize(from.size()); - for (size_t i = 0; i < from.size(); ++i) + size_t size = from.size(); + to.resize(size); + for (size_t i = 0; i < size; ++i) { to[i] = unalignedLoad(&(from.data()[i])); } From ffeee2b471b299daa7713522ff349222fdcfaa79 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 18:03:32 +0300 Subject: [PATCH 201/314] Remove T-Tests --- .../AggregateFunctionStudentTTest.cpp | 52 ---- .../AggregateFunctionStudentTTest.h | 262 ----------------- .../AggregateFunctionWelchTTest.cpp | 49 ---- .../AggregateFunctionWelchTTest.h | 274 ------------------ .../registerAggregateFunctions.cpp | 50 +++- .../registerAggregateFunctions.h | 47 --- src/AggregateFunctions/ya.make | 2 - .../0_stateless/01322_student_ttest.sql | 19 -- .../0_stateless/01322_ttest_scipy.python | 108 ------- .../queries/0_stateless/01322_ttest_scipy.sh | 8 - .../queries/0_stateless/01322_welch_ttest.sql | 37 --- 11 files changed, 48 insertions(+), 860 deletions(-) delete mode 100644 src/AggregateFunctions/AggregateFunctionStudentTTest.cpp delete mode 100644 src/AggregateFunctions/AggregateFunctionStudentTTest.h delete mode 100644 src/AggregateFunctions/AggregateFunctionWelchTTest.cpp delete mode 100644 src/AggregateFunctions/AggregateFunctionWelchTTest.h delete mode 100644 tests/queries/0_stateless/01322_student_ttest.sql delete mode 100644 tests/queries/0_stateless/01322_ttest_scipy.python delete mode 100755 tests/queries/0_stateless/01322_ttest_scipy.sh delete mode 100644 tests/queries/0_stateless/01322_welch_ttest.sql diff --git a/src/AggregateFunctions/AggregateFunctionStudentTTest.cpp b/src/AggregateFunctions/AggregateFunctionStudentTTest.cpp deleted file mode 100644 index 58fc9e5b5b9..00000000000 --- a/src/AggregateFunctions/AggregateFunctionStudentTTest.cpp +++ /dev/null @@ -1,52 +0,0 @@ -#include -#include -#include -#include "registerAggregateFunctions.h" - -#include -#include - - -// the return type is boolean (we use UInt8 as we do not have boolean in clickhouse) - -namespace ErrorCodes -{ -extern const int NOT_IMPLEMENTED; -} - -namespace DB -{ - -namespace -{ - -AggregateFunctionPtr createAggregateFunctionStudentTTest(const std::string & name, const DataTypes & argument_types, const Array & parameters) -{ - assertBinary(name, argument_types); - assertNoParameters(name, parameters); - - AggregateFunctionPtr res; - - if (isDecimal(argument_types[0]) || isDecimal(argument_types[1])) - { - throw Exception("Aggregate function " + name + " only supports numerical types", ErrorCodes::NOT_IMPLEMENTED); - } - else - { - res.reset(createWithTwoNumericTypes(*argument_types[0], *argument_types[1], argument_types)); - } - - if (!res) - { - throw Exception("Aggregate function " + name + " only supports numerical types", ErrorCodes::NOT_IMPLEMENTED); - } - - return res; -} -} - -void registerAggregateFunctionStudentTTest(AggregateFunctionFactory & factory) -{ - factory.registerFunction("studentTTest", createAggregateFunctionStudentTTest); -} -} diff --git a/src/AggregateFunctions/AggregateFunctionStudentTTest.h b/src/AggregateFunctions/AggregateFunctionStudentTTest.h deleted file mode 100644 index 0aef8f3ee2a..00000000000 --- a/src/AggregateFunctions/AggregateFunctionStudentTTest.h +++ /dev/null @@ -1,262 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -#if defined(OS_DARWIN) -extern "C" -{ - double lgammal_r(double x, int * signgamp); -} -#endif - - -namespace DB -{ - -template -struct AggregateFunctionStudentTTestData final -{ - size_t size_x = 0; - size_t size_y = 0; - X sum_x = static_cast(0); - Y sum_y = static_cast(0); - X square_sum_x = static_cast(0); - Y square_sum_y = static_cast(0); - Float64 mean_x = static_cast(0); - Float64 mean_y = static_cast(0); - - void add(X x, Y y) - { - sum_x += x; - sum_y += y; - size_x++; - size_y++; - mean_x = static_cast(sum_x) / size_x; - mean_y = static_cast(sum_y) / size_y; - square_sum_x += x * x; - square_sum_y += y * y; - } - - void merge(const AggregateFunctionStudentTTestData &other) - { - sum_x += other.sum_x; - sum_y += other.sum_y; - size_x += other.size_x; - size_y += other.size_y; - mean_x = static_cast(sum_x) / size_x; - mean_y = static_cast(sum_y) / size_y; - square_sum_x += other.square_sum_x; - square_sum_y += other.square_sum_y; - } - - void serialize(WriteBuffer &buf) const - { - writeBinary(mean_x, buf); - writeBinary(mean_y, buf); - writeBinary(sum_x, buf); - writeBinary(sum_y, buf); - writeBinary(square_sum_x, buf); - writeBinary(square_sum_y, buf); - writeBinary(size_x, buf); - writeBinary(size_y, buf); - } - - void deserialize(ReadBuffer &buf) - { - readBinary(mean_x, buf); - readBinary(mean_y, buf); - readBinary(sum_x, buf); - readBinary(sum_y, buf); - readBinary(square_sum_x, buf); - readBinary(square_sum_y, buf); - readBinary(size_x, buf); - readBinary(size_y, buf); - } - - size_t getSizeY() const - { - return size_y; - } - - size_t getSizeX() const - { - return size_x; - } - - Float64 getSSquared() const - { - /// The original formulae looks like - /// \frac{\sum_{i = 1}^{n_x}{(x_i - \bar{x}) ^ 2} + \sum_{i = 1}^{n_y}{(y_i - \bar{y}) ^ 2}}{n_x + n_y - 2} - /// But we made some mathematical transformations not to store original sequences. - /// Also we dropped sqrt, because later it will be squared later. - const Float64 all_x = square_sum_x + size_x * std::pow(mean_x, 2) - 2 * mean_x * sum_x; - const Float64 all_y = square_sum_y + size_y * std::pow(mean_y, 2) - 2 * mean_y * sum_y; - return static_cast(all_x + all_y) / (size_x + size_y - 2); - } - - - Float64 getTStatisticSquared() const - { - return std::pow(mean_x - mean_y, 2) / getStandartErrorSquared(); - } - - Float64 getTStatistic() const - { - return (mean_x - mean_y) / std::sqrt(getStandartErrorSquared()); - } - - Float64 getStandartErrorSquared() const - { - if (size_x == 0 || size_y == 0) - throw Exception("Division by zero encountered in Aggregate function StudentTTest", ErrorCodes::BAD_ARGUMENTS); - - return getSSquared() * (1.0 / static_cast(size_x) + 1.0 / static_cast(size_y)); - } - - Float64 getDegreesOfFreedom() const - { - return static_cast(size_x + size_y - 2); - } - - static Float64 integrateSimpson(Float64 a, Float64 b, std::function func) - { - const size_t iterations = std::max(1e6, 1e4 * std::abs(std::round(b))); - const long double h = (b - a) / iterations; - Float64 sum_odds = 0.0; - for (size_t i = 1; i < iterations; i += 2) - sum_odds += func(a + i * h); - Float64 sum_evens = 0.0; - for (size_t i = 2; i < iterations; i += 2) - sum_evens += func(a + i * h); - return (func(a) + func(b) + 2 * sum_evens + 4 * sum_odds) * h / 3; - } - - Float64 getPValue() const - { - const Float64 v = getDegreesOfFreedom(); - const Float64 t = getTStatisticSquared(); - auto f = [&v] (double x) { return std::pow(x, v/2 - 1) / std::sqrt(1 - x); }; - Float64 numenator = integrateSimpson(0, v / (t + v), f); - int unused; - Float64 denominator = std::exp(lgammal_r(v / 2, &unused) + lgammal_r(0.5, &unused) - lgammal_r(v / 2 + 0.5, &unused)); - return numenator / denominator; - } - - std::pair getResult() const - { - return std::make_pair(getTStatistic(), getPValue()); - } -}; - -/// Returns tuple of (t-statistic, p-value) -/// https://cpb-us-w2.wpmucdn.com/voices.uchicago.edu/dist/9/1193/files/2016/01/05b-TandP.pdf -template -class AggregateFunctionStudentTTest : - public IAggregateFunctionDataHelper,AggregateFunctionStudentTTest> -{ - -public: - AggregateFunctionStudentTTest(const DataTypes & arguments) - : IAggregateFunctionDataHelper, AggregateFunctionStudentTTest> ({arguments}, {}) - {} - - String getName() const override - { - return "studentTTest"; - } - - DataTypePtr getReturnType() const override - { - DataTypes types - { - std::make_shared>(), - std::make_shared>(), - }; - - Strings names - { - "t-statistic", - "p-value" - }; - - return std::make_shared( - std::move(types), - std::move(names) - ); - } - - void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override - { - auto col_x = assert_cast *>(columns[0]); - auto col_y = assert_cast *>(columns[1]); - - X x = col_x->getData()[row_num]; - Y y = col_y->getData()[row_num]; - - this->data(place).add(x, y); - } - - void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override - { - this->data(place).merge(this->data(rhs)); - } - - void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override - { - this->data(place).serialize(buf); - } - - void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override - { - this->data(place).deserialize(buf); - } - - void insertResultInto(AggregateDataPtr place, IColumn & to, Arena * /*arena*/) const override - { - size_t size_x = this->data(place).getSizeX(); - size_t size_y = this->data(place).getSizeY(); - - if (size_x < 2 || size_y < 2) - { - throw Exception("Aggregate function " + getName() + " requires samples to be of size > 1", ErrorCodes::BAD_ARGUMENTS); - } - - Float64 t_statistic = 0.0; - Float64 p_value = 0.0; - std::tie(t_statistic, p_value) = this->data(place).getResult(); - - /// Because p-value is a probability. - p_value = std::min(1.0, std::max(0.0, p_value)); - - auto & column_tuple = assert_cast(to); - auto & column_stat = assert_cast &>(column_tuple.getColumn(0)); - auto & column_value = assert_cast &>(column_tuple.getColumn(1)); - - column_stat.getData().push_back(t_statistic); - column_value.getData().push_back(p_value); - } - -}; - -}; diff --git a/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp b/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp deleted file mode 100644 index 0dcb125305d..00000000000 --- a/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp +++ /dev/null @@ -1,49 +0,0 @@ -#include -#include -#include -#include "registerAggregateFunctions.h" - -#include -#include - -namespace ErrorCodes -{ -extern const int NOT_IMPLEMENTED; -} - -namespace DB -{ - -namespace -{ - -AggregateFunctionPtr createAggregateFunctionWelchTTest(const std::string & name, const DataTypes & argument_types, const Array & parameters) -{ - assertBinary(name, argument_types); - assertNoParameters(name, parameters); - - AggregateFunctionPtr res; - - if (isDecimal(argument_types[0]) || isDecimal(argument_types[1])) - { - throw Exception("Aggregate function " + name + " only supports numerical types", ErrorCodes::NOT_IMPLEMENTED); - } - else - { - res.reset(createWithTwoNumericTypes(*argument_types[0], *argument_types[1], argument_types)); - } - - if (!res) - { - throw Exception("Aggregate function " + name + " only supports numerical types", ErrorCodes::NOT_IMPLEMENTED); - } - - return res; -} -} - -void registerAggregateFunctionWelchTTest(AggregateFunctionFactory & factory) -{ - factory.registerFunction("welchTTest", createAggregateFunctionWelchTTest); -} -} diff --git a/src/AggregateFunctions/AggregateFunctionWelchTTest.h b/src/AggregateFunctions/AggregateFunctionWelchTTest.h deleted file mode 100644 index b598f25162e..00000000000 --- a/src/AggregateFunctions/AggregateFunctionWelchTTest.h +++ /dev/null @@ -1,274 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -#if defined(OS_DARWIN) -extern "C" -{ - double lgammal_r(double x, int * signgamp); -} -#endif - - -namespace DB -{ - -template -struct AggregateFunctionWelchTTestData final -{ - size_t size_x = 0; - size_t size_y = 0; - X sum_x = static_cast(0); - Y sum_y = static_cast(0); - X square_sum_x = static_cast(0); - Y square_sum_y = static_cast(0); - Float64 mean_x = static_cast(0); - Float64 mean_y = static_cast(0); - - void add(X x, Y y) - { - sum_x += x; - sum_y += y; - size_x++; - size_y++; - mean_x = static_cast(sum_x) / size_x; - mean_y = static_cast(sum_y) / size_y; - square_sum_x += x * x; - square_sum_y += y * y; - } - - void merge(const AggregateFunctionWelchTTestData &other) - { - sum_x += other.sum_x; - sum_y += other.sum_y; - size_x += other.size_x; - size_y += other.size_y; - mean_x = static_cast(sum_x) / size_x; - mean_y = static_cast(sum_y) / size_y; - square_sum_x += other.square_sum_x; - square_sum_y += other.square_sum_y; - } - - void serialize(WriteBuffer &buf) const - { - writeBinary(mean_x, buf); - writeBinary(mean_y, buf); - writeBinary(sum_x, buf); - writeBinary(sum_y, buf); - writeBinary(square_sum_x, buf); - writeBinary(square_sum_y, buf); - writeBinary(size_x, buf); - writeBinary(size_y, buf); - } - - void deserialize(ReadBuffer &buf) - { - readBinary(mean_x, buf); - readBinary(mean_y, buf); - readBinary(sum_x, buf); - readBinary(sum_y, buf); - readBinary(square_sum_x, buf); - readBinary(square_sum_y, buf); - readBinary(size_x, buf); - readBinary(size_y, buf); - } - - size_t getSizeY() const - { - return size_y; - } - - size_t getSizeX() const - { - return size_x; - } - - Float64 getSxSquared() const - { - /// The original formulae looks like \frac{1}{size_x - 1} \sum_{i = 1}^{size_x}{(x_i - \bar{x}) ^ 2} - /// But we made some mathematical transformations not to store original sequences. - /// Also we dropped sqrt, because later it will be squared later. - return static_cast(square_sum_x + size_x * std::pow(mean_x, 2) - 2 * mean_x * sum_x) / (size_x - 1); - } - - Float64 getSySquared() const - { - /// The original formulae looks like \frac{1}{size_y - 1} \sum_{i = 1}^{size_y}{(y_i - \bar{y}) ^ 2} - /// But we made some mathematical transformations not to store original sequences. - /// Also we dropped sqrt, because later it will be squared later. - return static_cast(square_sum_y + size_y * std::pow(mean_y, 2) - 2 * mean_y * sum_y) / (size_y - 1); - } - - Float64 getTStatisticSquared() const - { - if (size_x == 0 || size_y == 0) - { - throw Exception("Division by zero encountered in Aggregate function WelchTTest", ErrorCodes::BAD_ARGUMENTS); - } - - return std::pow(mean_x - mean_y, 2) / (getSxSquared() / size_x + getSySquared() / size_y); - } - - Float64 getTStatistic() const - { - if (size_x == 0 || size_y == 0) - { - throw Exception("Division by zero encountered in Aggregate function WelchTTest", ErrorCodes::BAD_ARGUMENTS); - } - - return (mean_x - mean_y) / std::sqrt(getSxSquared() / size_x + getSySquared() / size_y); - } - - Float64 getDegreesOfFreedom() const - { - auto sx = getSxSquared(); - auto sy = getSySquared(); - Float64 numerator = std::pow(sx / size_x + sy / size_y, 2); - Float64 denominator_first = std::pow(sx, 2) / (std::pow(size_x, 2) * (size_x - 1)); - Float64 denominator_second = std::pow(sy, 2) / (std::pow(size_y, 2) * (size_y - 1)); - return numerator / (denominator_first + denominator_second); - } - - static Float64 integrateSimpson(Float64 a, Float64 b, std::function func) - { - size_t iterations = std::max(1e6, 1e4 * std::abs(std::round(b))); - double h = (b - a) / iterations; - Float64 sum_odds = 0.0; - for (size_t i = 1; i < iterations; i += 2) - sum_odds += func(a + i * h); - Float64 sum_evens = 0.0; - for (size_t i = 2; i < iterations; i += 2) - sum_evens += func(a + i * h); - return (func(a) + func(b) + 2 * sum_evens + 4 * sum_odds) * h / 3; - } - - Float64 getPValue() const - { - const Float64 v = getDegreesOfFreedom(); - const Float64 t = getTStatisticSquared(); - auto f = [&v] (double x) { return std::pow(x, v / 2 - 1) / std::sqrt(1 - x); }; - Float64 numenator = integrateSimpson(0, v / (t + v), f); - int unused; - Float64 denominator = std::exp(lgammal_r(v / 2, &unused) + lgammal_r(0.5, &unused) - lgammal_r(v / 2 + 0.5, &unused)); - return numenator / denominator; - } - - std::pair getResult() const - { - return std::make_pair(getTStatistic(), getPValue()); - } -}; - -/// Returns tuple of (t-statistic, p-value) -/// https://cpb-us-w2.wpmucdn.com/voices.uchicago.edu/dist/9/1193/files/2016/01/05b-TandP.pdf -template -class AggregateFunctionWelchTTest : - public IAggregateFunctionDataHelper,AggregateFunctionWelchTTest> -{ - -public: - AggregateFunctionWelchTTest(const DataTypes & arguments) - : IAggregateFunctionDataHelper, AggregateFunctionWelchTTest> ({arguments}, {}) - {} - - String getName() const override - { - return "welchTTest"; - } - - DataTypePtr getReturnType() const override - { - DataTypes types - { - std::make_shared>(), - std::make_shared>(), - }; - - Strings names - { - "t-statistic", - "p-value" - }; - - return std::make_shared( - std::move(types), - std::move(names) - ); - } - - void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override - { - auto col_x = assert_cast *>(columns[0]); - auto col_y = assert_cast *>(columns[1]); - - X x = col_x->getData()[row_num]; - Y y = col_y->getData()[row_num]; - - this->data(place).add(x, y); - } - - void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override - { - this->data(place).merge(this->data(rhs)); - } - - void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override - { - this->data(place).serialize(buf); - } - - void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override - { - this->data(place).deserialize(buf); - } - - void insertResultInto(AggregateDataPtr place, IColumn & to, Arena * /*arena*/) const override - { - size_t size_x = this->data(place).getSizeX(); - size_t size_y = this->data(place).getSizeY(); - - if (size_x < 2 || size_y < 2) - { - throw Exception("Aggregate function " + getName() + " requires samples to be of size > 1", ErrorCodes::BAD_ARGUMENTS); - } - - Float64 t_statistic = 0.0; - Float64 p_value = 0.0; - std::tie(t_statistic, p_value) = this->data(place).getResult(); - - /// Because p-value is a probability. - p_value = std::min(1.0, std::max(0.0, p_value)); - - auto & column_tuple = assert_cast(to); - auto & column_stat = assert_cast &>(column_tuple.getColumn(0)); - auto & column_value = assert_cast &>(column_tuple.getColumn(1)); - - column_stat.getData().push_back(t_statistic); - column_value.getData().push_back(p_value); - } - -}; - -}; diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 9fd02ba9d6c..90109a98433 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -7,6 +7,54 @@ namespace DB { +class AggregateFunctionFactory; +void registerAggregateFunctionAvg(AggregateFunctionFactory &); +void registerAggregateFunctionAvgWeighted(AggregateFunctionFactory &); +void registerAggregateFunctionCount(AggregateFunctionFactory &); +void registerAggregateFunctionGroupArray(AggregateFunctionFactory &); +void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory &); +void registerAggregateFunctionGroupArrayInsertAt(AggregateFunctionFactory &); +void registerAggregateFunctionsQuantile(AggregateFunctionFactory &); +void registerAggregateFunctionsSequenceMatch(AggregateFunctionFactory &); +void registerAggregateFunctionWindowFunnel(AggregateFunctionFactory &); +void registerAggregateFunctionRate(AggregateFunctionFactory &); +void registerAggregateFunctionsMinMaxAny(AggregateFunctionFactory &); +void registerAggregateFunctionsStatisticsStable(AggregateFunctionFactory &); +void registerAggregateFunctionsStatisticsSimple(AggregateFunctionFactory &); +void registerAggregateFunctionSum(AggregateFunctionFactory &); +void registerAggregateFunctionSumMap(AggregateFunctionFactory &); +void registerAggregateFunctionsUniq(AggregateFunctionFactory &); +void registerAggregateFunctionUniqCombined(AggregateFunctionFactory &); +void registerAggregateFunctionUniqUpTo(AggregateFunctionFactory &); +void registerAggregateFunctionTopK(AggregateFunctionFactory &); +void registerAggregateFunctionsBitwise(AggregateFunctionFactory &); +void registerAggregateFunctionsBitmap(AggregateFunctionFactory &); +void registerAggregateFunctionsMaxIntersections(AggregateFunctionFactory &); +void registerAggregateFunctionHistogram(AggregateFunctionFactory &); +void registerAggregateFunctionRetention(AggregateFunctionFactory &); +void registerAggregateFunctionTimeSeriesGroupSum(AggregateFunctionFactory &); +void registerAggregateFunctionMLMethod(AggregateFunctionFactory &); +void registerAggregateFunctionEntropy(AggregateFunctionFactory &); +void registerAggregateFunctionSimpleLinearRegression(AggregateFunctionFactory &); +void registerAggregateFunctionMoving(AggregateFunctionFactory &); +void registerAggregateFunctionCategoricalIV(AggregateFunctionFactory &); +void registerAggregateFunctionAggThrow(AggregateFunctionFactory &); +void registerAggregateFunctionWelchTTest(AggregateFunctionFactory &); +void registerAggregateFunctionStudentTTest(AggregateFunctionFactory &); +void registerAggregateFunctionRankCorrelation(AggregateFunctionFactory &); + +class AggregateFunctionCombinatorFactory; +void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &); +void registerAggregateFunctionCombinatorArray(AggregateFunctionCombinatorFactory &); +void registerAggregateFunctionCombinatorForEach(AggregateFunctionCombinatorFactory &); +void registerAggregateFunctionCombinatorState(AggregateFunctionCombinatorFactory &); +void registerAggregateFunctionCombinatorMerge(AggregateFunctionCombinatorFactory &); +void registerAggregateFunctionCombinatorNull(AggregateFunctionCombinatorFactory &); +void registerAggregateFunctionCombinatorOrFill(AggregateFunctionCombinatorFactory &); +void registerAggregateFunctionCombinatorResample(AggregateFunctionCombinatorFactory &); +void registerAggregateFunctionCombinatorDistinct(AggregateFunctionCombinatorFactory &); + + void registerAggregateFunctions() { { @@ -45,8 +93,6 @@ void registerAggregateFunctions() registerAggregateFunctionMoving(factory); registerAggregateFunctionCategoricalIV(factory); registerAggregateFunctionAggThrow(factory); - registerAggregateFunctionWelchTTest(factory); - registerAggregateFunctionStudentTTest(factory); registerAggregateFunctionRankCorrelation(factory); } diff --git a/src/AggregateFunctions/registerAggregateFunctions.h b/src/AggregateFunctions/registerAggregateFunctions.h index abbba56ed32..2a2e0bb7d3f 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.h +++ b/src/AggregateFunctions/registerAggregateFunctions.h @@ -3,53 +3,6 @@ namespace DB { -class AggregateFunctionFactory; -void registerAggregateFunctionAvg(AggregateFunctionFactory &); -void registerAggregateFunctionAvgWeighted(AggregateFunctionFactory &); -void registerAggregateFunctionCount(AggregateFunctionFactory &); -void registerAggregateFunctionGroupArray(AggregateFunctionFactory &); -void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory &); -void registerAggregateFunctionGroupArrayInsertAt(AggregateFunctionFactory &); -void registerAggregateFunctionsQuantile(AggregateFunctionFactory &); -void registerAggregateFunctionsSequenceMatch(AggregateFunctionFactory &); -void registerAggregateFunctionWindowFunnel(AggregateFunctionFactory &); -void registerAggregateFunctionRate(AggregateFunctionFactory &); -void registerAggregateFunctionsMinMaxAny(AggregateFunctionFactory &); -void registerAggregateFunctionsStatisticsStable(AggregateFunctionFactory &); -void registerAggregateFunctionsStatisticsSimple(AggregateFunctionFactory &); -void registerAggregateFunctionSum(AggregateFunctionFactory &); -void registerAggregateFunctionSumMap(AggregateFunctionFactory &); -void registerAggregateFunctionsUniq(AggregateFunctionFactory &); -void registerAggregateFunctionUniqCombined(AggregateFunctionFactory &); -void registerAggregateFunctionUniqUpTo(AggregateFunctionFactory &); -void registerAggregateFunctionTopK(AggregateFunctionFactory &); -void registerAggregateFunctionsBitwise(AggregateFunctionFactory &); -void registerAggregateFunctionsBitmap(AggregateFunctionFactory &); -void registerAggregateFunctionsMaxIntersections(AggregateFunctionFactory &); -void registerAggregateFunctionHistogram(AggregateFunctionFactory &); -void registerAggregateFunctionRetention(AggregateFunctionFactory &); -void registerAggregateFunctionTimeSeriesGroupSum(AggregateFunctionFactory &); -void registerAggregateFunctionMLMethod(AggregateFunctionFactory &); -void registerAggregateFunctionEntropy(AggregateFunctionFactory &); -void registerAggregateFunctionSimpleLinearRegression(AggregateFunctionFactory &); -void registerAggregateFunctionMoving(AggregateFunctionFactory &); -void registerAggregateFunctionCategoricalIV(AggregateFunctionFactory &); -void registerAggregateFunctionAggThrow(AggregateFunctionFactory &); -void registerAggregateFunctionWelchTTest(AggregateFunctionFactory &); -void registerAggregateFunctionStudentTTest(AggregateFunctionFactory &); -void registerAggregateFunctionRankCorrelation(AggregateFunctionFactory &); - -class AggregateFunctionCombinatorFactory; -void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &); -void registerAggregateFunctionCombinatorArray(AggregateFunctionCombinatorFactory &); -void registerAggregateFunctionCombinatorForEach(AggregateFunctionCombinatorFactory &); -void registerAggregateFunctionCombinatorState(AggregateFunctionCombinatorFactory &); -void registerAggregateFunctionCombinatorMerge(AggregateFunctionCombinatorFactory &); -void registerAggregateFunctionCombinatorNull(AggregateFunctionCombinatorFactory &); -void registerAggregateFunctionCombinatorOrFill(AggregateFunctionCombinatorFactory &); -void registerAggregateFunctionCombinatorResample(AggregateFunctionCombinatorFactory &); -void registerAggregateFunctionCombinatorDistinct(AggregateFunctionCombinatorFactory &); - void registerAggregateFunctions(); } diff --git a/src/AggregateFunctions/ya.make b/src/AggregateFunctions/ya.make index 8d27cf986d0..80b8de0eea7 100644 --- a/src/AggregateFunctions/ya.make +++ b/src/AggregateFunctions/ya.make @@ -41,7 +41,6 @@ SRCS( AggregateFunctionState.cpp AggregateFunctionStatistics.cpp AggregateFunctionStatisticsSimple.cpp - AggregateFunctionStudentTTest.cpp AggregateFunctionSum.cpp AggregateFunctionSumMap.cpp AggregateFunctionTimeSeriesGroupSum.cpp @@ -49,7 +48,6 @@ SRCS( AggregateFunctionUniqCombined.cpp AggregateFunctionUniq.cpp AggregateFunctionUniqUpTo.cpp - AggregateFunctionWelchTTest.cpp AggregateFunctionWindowFunnel.cpp parseAggregateFunctionParameters.cpp registerAggregateFunctions.cpp diff --git a/tests/queries/0_stateless/01322_student_ttest.sql b/tests/queries/0_stateless/01322_student_ttest.sql deleted file mode 100644 index b8b86384bc4..00000000000 --- a/tests/queries/0_stateless/01322_student_ttest.sql +++ /dev/null @@ -1,19 +0,0 @@ -DROP TABLE IF EXISTS student_ttest; - -/*Check t-stat and p-value and compare it with scipy.stat implementation - First: a=1, sigma (not sigma^2)=5, size=500 - Second: a=1, sigma = 5, size = 500 */ -CREATE TABLE student_ttest (left Float64, right Float64) ENGINE = Memory; -INSERT INTO student_ttest VALUES (0.88854,-2.90702), (-5.76966,3.61651), (6.76618,4.27458), (3.55546,4.82133), (-9.76948,9.59483), (4.92323,1.00424), (-0.36352,2.04147), (0.97018,-3.58214), (4.61656,6.59543), (-6.78292,-1.00532), (4.02008,-3.59794), (12.41838,-2.82434), (5.14417,-3.13194), (3.86836,9.90977), (-1.26199,0.523), (12.44106,4.62779), (3.28349,-2.56872), (1.77261,2.25807), (-8.94748,1.04044), (-1.01449,-2.35744), (-1.26377,10.81531), (6.79682,-9.68469), (6.32333,3.80885), (-8.21214,12.70435), (-1.68565,-6.01112), (9.7557,1.89065), (3.66694,5.08892), (1.39967,3.45254), (-5.52035,11.58151), (-10.95601,0.85035), (0.93877,8.38397), (1.45933,1.17169), (-5.40551,4.74621), (-0.83857,-1.66614), (8.50794,4.2414), (-6.68686,1.68765), (5.03099,1.85223), (1.56251,9.10111), (4.17381,-2.38085), (-2.92644,-14.79595), (5.11068,-3.8938), (2.09617,-3.41864), (11.7787,-3.15282), (6.50336,-0.56684), (0.62098,12.87997), (-7.97121,6.89115), (3.81902,12.921), (0.33151,-7.94908), (10.68584,2.45687), (0.56007,2.14957), (-7.38621,7.55081), (5.05882,-3.71534), (2.34616,-2.41064), (11.3806,-0.80734), (5.95276,-4.75651), (-3.01429,2.05241), (5.98169,-5.44523), (0.96985,-2.75054), (-1.15932,-13.00131), (2.11547,-2.74451), (2.49668,-1.39004), (-12.49569,-3.02854), (-4.94667,7.65112), (-3.64215,1.1245), (-8.35595,6.74117), (3.211,-0.75777), (2.33805,8.93451), (2.38608,-8.85559), (-3.2862,-0.36405), (-0.80454,4.02742), (-0.53483,6.88718), (10.66445,-1.05124), (-0.37619,3.04085), (0.48246,3.32368), (7.41919,1.147), (0.42414,3.41554), (-2.32335,-3.47851), (-0.70223,-0.47684), (-5.9332,-0.55605), (-1.20561,-0.17006), (3.39865,2.26218), (9.61739,12.45494), (-0.78651,-1.84097), (-4.00256,1.64934), (-7.99646,-7.07496), (8.72923,-9.99462), (0.71859,6.09954), (-1.62726,-1.05319), (5.11234,3.04757), (-0.95625,0.93899), (-3.75573,-4.63243), (1.03141,-7.43322), (-3.33588,-7.298), (1.51804,-6.59016), (-3.30935,-6.11649), (-1.97507,0.56682), (4.06456,2.00661), (3.27195,-2.79814), (-7.81761,2.84482), (-3.81785,3.65348), (-4.18311,-4.22807), (-11.33313,-4.54336), (-0.25221,-3.63343), (7.2514,2.96878), (5.30301,6.11661), (2.46762,-1.70919), (4.22716,-4.71133), (0.33916,6.09652), (9.7638,-6.83454), (-7.58684,0.18006), (-4.09888,1.51676), (4.26617,-5.31646), (-0.56744,-3.21215), (4.65125,-5.07599), (-1.30301,-2.36591), (4.53771,3.55724), (9.96929,4.8904), (3.72939,-3.22586), (-2.29818,-1.74928), (3.09417,5.73458), (0.82251,1.41188), (5.29975,2.86255), (2.8685,2.90179), (-5.73321,-2.19949), (-1.85651,1.72727), (-1.07984,1.76939), (9.78342,-0.12848), (-13.49652,-0.52), (3.68791,3.48333), (1.9998,7.8262), (1.11674,0.09099), (9.43869,7.77017), (4.07029,9.49484), (5.32715,1.42825), (7.16504,1.99624), (6.66096,4.00419), (-5.7111,1.07925), (-0.38575,-0.09987), (4.49165,-5.48733), (-3.36489,-1.83517), (7.71814,2.38059), (-1.58966,1.42075), (-1.61063,-1.11968), (-0.91602,-6.46035), (0.73459,7.66576), (-3.24463,4.6307), (6.3947,5.55989), (-2.77845,3.16684), (4.45899,5.07671), (-8.84186,-10.20566), (2.62276,-4.73386), (1.774,1.28353), (4.3692,6.75679), (0.05942,12.09895), (-1.44042,7.0049), (-2.53594,7.16156), (-2.24752,-0.64311), (4.98874,-0.66747), (4.05434,3.99996), (-2.56483,9.07298), (-6.79286,-4.60971), (-2.06165,0.70744), (-0.26056,2.56774), (1.89567,9.32424), (-3.15145,3.95087), (-7.31321,7.11372), (0.28936,-0.89284), (-0.63111,8.6155), (0.22611,-0.14141), (-9.3377,-4.86319), (-5.76638,-6.95801), (3.87306,4.44883), (6.7011,4.6156), (9.03915,-2.3579), (-1.21835,-5.1186), (0.82892,8.12819), (2.80656,2.78392), (-1.34746,-4.30221), (-1.99912,-1.47506), (0.6036,6.8598), (-3.46117,0.47636), (5.23732,0.95383), (-1.86702,7.79779), (-5.86115,-2.61767), (6.48523,-10.5087), (-7.40158,-2.74299), (-1.38913,3.87369), (4.94613,-1.07093), (-2.07818,4.98864), (2.39808,-7.50772), (4.89238,6.41316), (4.39481,1.39061), (5.20425,-3.1747), (13.62598,-2.13621), (-2.86293,-0.02203), (-3.62396,0.89025), (-4.28695,-5.87746), (4.66425,3.60026), (2.20871,-0.23178), (1.60382,-2.1897), (-9.87024,-5.85101), (-7.37302,-1.6053), (-4.17814,3.6184), (2.5148,-8.53795), (3.21708,-0.35987), (-11.48089,2.15301), (1.19821,-6.60692), (-0.07436,9.54341), (-1.10652,1.11511), (4.03395,2.94025), (-4.35883,12.05657), (2.04013,3.75156), (0.52264,7.95597), (8.14004,-0.99449), (-8.86949,0.90597), (-0.35807,-7.90627), (-10.71113,3.50863), (-2.13755,-1.47493), (0.50715,4.11671), (6.30826,10.06325), (2.37527,-1.06059), (0.20872,-1.37737), (-5.85729,-0.42542), (-4.97217,-3.90267), (-9.78434,9.35037), (-1.53277,-7.91219), (0.14827,-4.69945), (-1.053,3.63776), (1.74558,3.46492), (11.17194,2.84518), (9.35487,-3.04301), (-9.17209,8.82764), (10.41814,7.80134), (7.41206,7.87755), (3.71775,7.01035), (-2.04674,2.43271), (6.18037,11.36418), (5.6383,-6.92659), (-0.90058,5.95541), (-1.27073,3.59436), (-2.3473,5.18429), (-8.44271,4.20225), (2.75551,0.5029), (-1.15521,4.03074), (4.08722,5.23152), (-1.70399,10.65409), (7.24114,-0.69845), (-8.43976,11.70096), (-1.53052,5.80692), (-0.00526,-8.1819), (-4.04813,4.31485), (-2.84299,5.7227), (-5.201,5.67398), (7.75774,-1.75826), (-2.85791,7.54164), (-3.86071,-1.79026), (-1.80029,-1.7395), (-5.26015,5.65042), (-3.158,0.38765), (7.71014,-4.64719), (-4.84866,-10.22048), (-8.38785,-2.05447), (7.67021,-2.43441), (4.96521,-5.38551), (-0.40919,5.47764), (-3.25711,8.26637), (3.07685,-3.6421), (2.89376,-11.66269), (-10.47331,3.972), (-3.48942,5.46642), (1.13906,-3.72304), (-8.57454,5.75251), (-3.38963,5.12841), (-2.3195,0.59067), (-1.60694,5.21138), (-5.57406,-4.58702), (-0.93075,-8.737), (-11.76579,-2.12737), (10.68283,0.22888), (8.74324,-1.46448), (7.66409,2.40311), (4.76715,-5.21814), (0.44539,13.94749), (-1.35941,-2.77448), (4.18849,-3.7867), (-6.17097,3.4954), (0.27977,3.12586), (-1.45006,-7.01485), (-4.81694,-3.20727), (-3.0297,6.31415), (0.02145,2.37521), (2.46883,8.13787), (9.60317,2.15956), (-9.93898,-0.40842), (1.05549,-7.27283), (5.55366,4.27575), (-3.80722,-2.89126), (-4.18851,6.84344), (1.00351,7.0869), (3.11385,-5.18837), (-5.17623,2.67648), (-3.18396,-6.57021), (-6.65302,0.60429), (-0.50832,-1.04921), (-4.04375,7.12873), (4.52707,1.68973), (6.63124,-2.58404), (-3.72082,-3.83114), (5.79825,-7.26546), (-2.0158,-5.07153), (-2.78369,-0.80395), (-1.91821,2.09455), (6.31714,4.33374), (-1.80869,8.54335), (8.55586,0.80566), (2.40826,-8.38085), (-8.46361,7.54812), (5.04452,8.78007), (-0.84665,1.5857), (2.30903,8.43855), (-3.71837,-1.90846), (-0.69419,-1.2434), (3.6733,7.16172), (-1.96098,-3.44129), (2.36747,-6.37542), (-12.03622,-4.99486), (4.38481,4.99033), (2.93955,-1.83734), (2.16804,-2.83289), (-0.08218,-4.13997), (-3.97934,1.40163), (-7.43985,8.57867), (0.91666,-1.87639), (7.23432,3.41667), (-6.13303,6.31762), (-10.23217,1.58473), (-6.21681,1.63625), (-0.80934,-6.93618), (0.17914,3.58046), (2.13338,-6.8097), (6.97656,4.69978), (6.90455,-1.72912), (6.25943,5.29491), (-6.04019,-1.63062), (-7.30909,5.83818), (1.4589,17.0769), (12.00208,4.54301), (2.22457,-1.33801), (-2.45912,5.64339), (-6.92213,1.26913), (4.05547,-1.01553), (0.04709,4.8316), (-7.70952,3.08635), (-1.47883,-2.27738), (1.3701,-1.13761), (-4.92928,10.08698), (-2.75872,5.33827), (-0.09178,2.84345), (2.62642,-1.51132), (-1.14623,13.46078), (2.76609,8.58965), (4.94404,-2.36683), (-7.01764,-1.8217), (-10.91568,1.96981), (-2.49738,2.31718), (0.73576,3.66493), (2.25436,1.93104), (-1.72956,5.20332), (2.41054,3.20519), (5.72149,3.34631), (-6.41371,7.0087), (3.38217,-7.96126), (1.24133,-0.62182), (10.03634,-4.65227), (-2.37303,10.6572), (-1.35543,4.50891), (-1.4387,9.74298), (-4.0976,3.85707), (-0.82501,6.41144), (-1.93498,1.48649), (5.59955,2.28076), (5.46656,2.75342), (2.43568,-5.40401), (-0.23926,7.11389), (-4.9945,5.74368), (-4.96655,6.78345), (-0.59258,3.83773), (2.02497,0.70959), (0.67583,0.57434), (3.16522,1.5888), (-1.9673,3.94889), (-6.75319,5.8234), (-6.69723,7.78366), (0.81148,9.08354), (4.44531,-7.99182), (-4.43522,-2.77033), (-5.28602,-10.29342), (-3.58829,1.76251), (-7.97395,2.09266), (-2.84891,4.20614), (-3.95112,-3.63064), (3.54945,-2.17794), (12.12376,-2.66225), (-3.12347,-2.74707), (3.65209,-1.93431), (9.34031,1.38629), (-0.26348,4.12816), (-5.23968,-1.58902), (2.22336,-5.08864), (-10.70405,-2.30491), (-4.41319,2.64605), (-5.94912,1.16158), (1.8147,2.63534), (7.69287,1.4956), (9.46125,-4.60768), (4.72497,0.60771), (-0.57565,3.29549), (-1.12303,-1.42592), (2.90272,0.8883), (-4.4584,-1.10612), (4.28819,-2.57296), (11.64512,5.88085), (-1.80395,7.40745), (2.51605,13.48116), (-3.18439,5.53539), (-0.70213,-1.46014), (-7.68383,3.73304), (-8.32268,3.5435), (-8.71115,-3.89151), (9.96933,4.16265), (0.95675,2.32663), (3.35114,5.31735), (-2.66008,6.33485), (7.75456,2.1339), (0.73568,0.82708), (0.3483,-2.95155), (-1.09203,-6.76019), (-7.76963,-4.20179), (5.81902,8.78354), (-3.41424,1.41863), (-0.39209,7.65689), (4.67608,-6.52601), (0.68753,-4.4426), (5.17179,-4.49483), (4.98983,-3.91479), (-0.12659,-2.84562), (3.25267,2.58974), (1.50184,2.24424), (2.94507,-4.65846), (-0.42333,8.4062), (-3.66227,8.20262), (8.90812,-8.63752), (4.74411,4.97966), (2.22018,-0.35563), (-2.07976,-4.72116), (4.8711,-2.95997), (0.5023,2.73959), (6.31569,-0.23956), (-4.36903,10.13915), (3.82146,11.83775), (-6.99477,-2.50332), (3.61225,-0.58181), (14.69335,-7.62836), (0.58368,2.26478), (4.65341,-3.50179), (-3.14272,-2.08023), (2.67048,4.07256), (4.64963,-1.40826), (-2.70828,-2.33644), (1.42923,3.00197), (5.84498,4.23668), (-4.76568,-2.24647), (0.19907,1.0445), (1.67486,-0.31901), (5.32145,8.62657), (-8.03477,3.92817), (3.46776,0.08462), (4.66374,10.15884), (-5.37394,0.4113), (5.39045,4.45847), (-1.44756,5.82941), (-1.64419,6.59202), (3.39699,-3.73441), (-2.94659,-5.86969), (-2.38437,-4.56543), (-0.23958,-1.32636), (6.88389,-0.17884), (-2.7172,-3.56181), (-1.53419,-0.66932), (7.38841,6.87538), (-5.44178,0.73527), (-0.89287,-0.24177), (2.93546,-0.8657), (-0.26901,-0.22977), (-4.70044,1.02095), (2.25846,6.16311), (-9.28813,-5.68027), (6.04268,-3.7619), (4.41693,4.22959), (1.75714,-1.5249); -SELECT '-2.610898982580138', '0.00916587538237954'; -SELECT roundBankers(studentTTest(left, right).1, 16) as t_stat, roundBankers(studentTTest(left, right).2, 16) as p_value from student_ttest; -DROP TABLE IF EXISTS student_ttest; - -/*Check t-stat and p-value and compare it with scipy.stat implementation - First: a=1, sigma (not sigma^2)=5, size=500 - Second: a=1, sigma = 5, size = 500 */ -CREATE TABLE student_ttest (left Float64, right Float64) ENGINE = Memory; -INSERT INTO student_ttest VALUES (4.52546,8.69444), (3.73628,3.81414), (-0.39478,12.38442), (5.15633,8.9738), (0.50539,9.19594), (-5.34036,7.21009), (0.19336,4.97743), (8.35729,4.94756), (6.95818,19.80911), (-2.93812,13.75358), (8.30807,16.56373), (-3.3517,9.72882), (4.16279,4.64509), (-3.17231,17.76854), (1.93545,4.80693), (11.06606,8.79505), (-4.22678,10.88868), (-1.99975,6.21932), (-4.51178,15.11614), (-4.50711,13.24703), (1.89786,14.76476), (-6.19638,-0.6117), (-3.70188,17.48993), (5.01334,12.11847), (1.79036,4.87439), (2.14435,18.56479), (3.0282,1.23712), (2.35528,5.41596), (-12.18535,4.54994), (5.59709,11.37668), (-12.92336,9.5982), (-0.04281,6.59822), (-0.16923,1.16703), (0.88924,8.88418), (-4.68414,10.95047), (8.01099,5.52787), (2.61686,-1.11647), (-2.76895,14.49946), (3.32165,3.27585), (-0.85135,-0.42025), (1.21368,6.37906), (4.38673,2.5242), (6.20964,8.1405), (-1.23172,6.46732), (4.65516,9.89332), (-1.87143,10.4374), (0.86429,-1.06465), (2.51184,6.84902), (-1.88822,10.96576), (-1.61802,7.83319), (1.93653,14.39823), (-3.66631,7.02594), (-1.05294,13.46629), (-10.74718,10.39531), (16.49295,11.27348), (-7.65494,9.32187), (-3.39303,12.32667), (-4.89418,8.98905), (3.2521,9.54757), (0.05831,5.98325), (-3.00409,3.47248), (5.76702,9.26966), (2.67674,5.77816), (10.52623,6.32966), (-0.54501,9.49313), (-4.89835,6.21337), (3.52457,10.00242), (-0.0451,6.25167), (-6.61226,15.64671), (9.02391,2.78968), (5.52571,6.55442), (4.54352,3.68819), (-3.8394,9.55934), (-7.75295,4.166), (5.91167,12.32471), (1.38897,7.10969), (6.24166,16.31723), (5.58536,12.99482), (4.7591,10.11585), (-2.58336,10.29455), (-1.91263,18.27524), (3.31575,12.84435), (5.3507,13.11954), (-15.22081,12.84147), (-0.84775,15.55658), (-4.538,11.45329), (6.71177,7.50912), (0.52882,8.56226), (2.0242,8.63104), (5.69146,15.68026), (4.63328,21.6361), (0.22984,6.23925), (-2.84052,8.65714), (7.91867,9.9423), (1.11001,12.28213), (-0.11251,3.11279), (-0.20905,13.58128), (0.03287,16.51407), (-1.59397,16.60476), (-5.39405,12.02022), (-7.1233,12.11035), (4.51517,9.47832), (-0.70967,6.40742), (5.67299,8.87252), (-0.33835,15.14265), (-1.83047,2.23572), (-0.62877,11.57144), (-7.23148,18.87737), (0.1802,12.1833), (11.73325,11.17519), (2.17603,16.80422), (-0.11683,6.81423), (-1.29102,12.12546), (-0.23201,8.06153), (-6.8643,10.97228), (-6.85153,7.30596), (-4.77163,15.44026), (6.11721,8.00993), (5.96406,12.60196), (3.59135,13.96832), (-0.60095,14.03207), (3.11163,4.53758), (-0.18831,8.08297), (0.67657,4.90451), (-3.16117,8.14253), (0.26957,19.88605), (2.18653,13.85254), (-5.94611,23.01839), (-4.39352,6.02084), (-3.71525,9.60319), (5.11103,1.90511), (1.33998,10.35237), (1.01629,16.27082), (-3.36917,12.52379), (-3.99661,11.37435), (8.19336,13.61823), (2.89168,15.77622), (-11.10373,15.17254), (11.68005,6.711), (3.08282,4.74205), (-6.81506,10.09812), (-2.34587,6.61722), (-2.68725,10.34164), (0.3577,8.96602), (-3.05682,12.32157), (9.08062,11.75711), (-0.77913,13.49499), (10.35215,8.57713), (6.82565,11.50313), (-1.24674,1.13097), (5.18822,7.83205), (-3.70743,5.77957), (1.40319,15.5519), (5.89432,10.82676), (1.43152,11.51218), (6.70638,9.29779), (9.76613,9.77021), (4.27604,9.94114), (-2.63141,15.54513), (-7.8133,19.10736), (-0.06668,15.04205), (1.05391,9.03114), (4.41797,24.0104), (0.09337,9.94205), (6.16075,2.5925), (7.49413,8.82726), (-3.52872,10.0209), (-2.17126,8.1635), (-3.87605,4.24074), (3.26607,7.67291), (-3.28045,5.21642), (2.1429,11.2808), (1.53386,6.88172), (0.21169,5.98743), (-0.63674,17.97249), (5.84893,6.46323), (-0.63498,15.37416), (8.29526,2.89957), (-1.08358,17.13044), (-2.306,11.06355), (2.86991,3.09625), (-0.76074,-2.33019), (5.49191,7.42675), (1.82883,15.06792), (-3.70497,8.81116), (-0.53232,19.17446), (-11.49722,18.77181), (3.44877,14.06443), (-1.8596,12.81241), (-10.34851,2.72299), (1.13093,18.67739), (-10.93389,11.63275), (-3.39703,2.23891), (0.19749,13.01195), (-3.68389,7.43402), (-4.67863,8.14599), (10.78916,16.65328), (0.37675,1.362), (3.98094,3.87957), (-3.64775,11.16134), (-4.8443,6.25357), (1.102,4.21945), (8.72112,12.50047), (-1.47361,6.45486), (6.24183,18.99924), (6.83569,18.09508), (-3.11684,13.59528), (4.91306,3.39681), (-0.03628,13.33157), (5.1282,5.8945), (-2.38558,5.61212), (2.33351,8.41149), (-0.97191,13.78608), (-0.05588,6.08609), (-4.70019,12.76962), (-5.12371,3.26206), (0.65606,0.25528), (-0.11574,11.9083), (4.4238,4.35071), (6.93399,11.19855), (3.68712,13.87404), (-0.01187,6.87986), (1.8332,8.32566), (5.81322,22.51334), (-4.04709,2.5226), (-8.26397,16.84498), (-2.11273,6.26108), (5.28396,13.84824), (0.73054,6.03262), (6.43559,14.12668), (4.35565,16.01939), (-1.05545,8.19237), (5.00087,18.01595), (-2.72239,9.45609), (7.32313,6.90459), (2.11548,12.83115), (-3.40953,10.603), (6.97051,13.70439), (-0.45567,6.1633), (1.31699,4.1151), (-1.49871,8.20499), (7.14772,11.67903), (0.79277,7.30851), (6.9698,6.50941), (2.08733,7.3949), (-3.55962,12.80075), (0.75601,5.62043), (1.21,18.2542), (-2.17877,17.9393), (1.83206,16.4569), (5.72463,8.78811), (7.42257,4.85949), (0.97829,-3.36394), (7.54238,5.38683), (9.91081,12.26083), (-4.61743,10.27907), (-4.40799,11.5144), (9.99854,11.57335), (8.53725,1.94203), (3.2905,7.78228), (0.38634,11.79385), (-2.53374,10.18415), (4.94758,14.67613), (4.79624,4.70301), (5.57664,12.72151), (-6.44871,-3.35508), (3.34431,17.63775), (0.14209,2.53883), (10.88431,14.01483), (0.31846,12.4387), (-0.54703,11.15408), (-4.67791,7.74882), (-5.68011,13.60956), (-4.93362,7.81991), (1.2271,10.90969), (5.27512,8.19828), (-3.84611,-1.18523), (6.81706,0.5916), (10.33033,0.35805), (5.13979,12.98364), (3.66534,11.38628), (-2.07219,13.94644), (10.65442,2.03781), (-3.31751,10.74447), (-1.82011,12.35656), (-0.39886,7.08701), (1.77052,2.69871), (1.29049,19.66653), (7.92344,7.88636), (-2.92595,10.36916), (-2.67107,1.632), (5.64708,11.86081), (0.34639,13.47602), (-3.04356,6.60204), (3.98828,7.01303), (-1.36695,20.19992), (-8.48462,18.88249), (-4.04669,11.34367), (9.84561,12.97305), (-6.1537,9.5776), (0.82433,17.91364), (1.92449,18.3247), (2.51288,9.9211), (0.40965,7.14257), (2.89183,6.59133), (3.84347,12.35274), (0.66829,10.57523), (-3.45094,12.12859), (1.3544,9.47177), (-9.85456,0.60659), (5.25689,4.72996), (-5.26018,4.51121), (-6.16912,13.28893), (-1.77163,8.09014), (3.96687,8.02511), (0.70893,13.85406), (-5.45342,1.75412), (-3.89706,6.00641), (3.11868,6.35554), (4.41714,7.11293), (7.64841,8.30442), (0.00489,12.63024), (3.2263,12.38966), (-5.33042,7.6801), (2.52189,11.33744), (-7.40308,4.67713), (0.67891,7.62276), (2.49343,2.14478), (5.43133,15.32988), (-0.67541,1.52299), (-0.60299,17.00017), (-6.32903,8.29701), (-3.44336,10.92961), (-0.23963,6.78449), (6.94686,7.02698), (6.59442,11.51719), (-4.18532,9.97926), (-1.8228,7.44251), (-0.29443,7.58541), (2.99821,4.76058), (2.51942,12.88959), (-3.49176,9.974), (-0.57979,17.03689), (8.69471,11.14554), (-1.19427,11.7392), (-3.17119,11.50029), (-2.99566,19.41759), (-3.34493,9.65127), (-2.33826,9.87673), (-5.04164,14.13485), (-0.48214,9.78034), (7.45097,1.57826), (3.04787,3.72091), (2.92632,9.4054), (1.39694,23.22816), (4.38686,-0.12571), (3.25753,6.97343), (7.14218,10.09049), (-4.04341,11.78393), (-9.19352,3.01909), (2.78473,16.09448), (0.33331,6.25485), (9.89238,7.13164), (6.00566,7.75879), (-1.7511,9.56834), (4.77815,6.14824), (5.07457,13.53454), (2.56132,8.26364), (2.38317,8.7095), (-1.63486,10.61607), (-1.46871,10.64418), (-5.8681,23.9106), (-2.96227,11.38978), (-1.90638,11.4383), (-13.3052,18.41498), (-2.14705,3.70959), (-9.62069,19.95918), (2.29313,9.53847), (0.22162,14.04957), (-1.83956,13.70151), (4.1853,5.45046), (6.05965,10.95061), (-0.23737,9.55156), (6.07452,17.92345), (4.34629,6.23976), (4.02922,8.71029), (3.62622,13.58736), (-3.95825,8.78527), (-1.63412,11.14213), (-1.25727,12.23717), (5.06323,16.44557), (-0.66176,0.47144), (2.36606,9.7198), (-5.77792,13.50981), (4.535,14.27806), (1.02031,13.50793), (4.49345,7.47381), (-4.99791,11.07844), (2.46716,9.89844), (3.65471,21.48548), (11.2283,6.92085), (6.69743,4.44074), (-5.60375,19.98074), (0.28683,7.92826), (-0.85737,16.6313), (4.26726,17.17618), (-3.4322,13.80807), (-2.07039,5.37083), (-2.26798,9.73962), (-0.99818,10.66273), (0.41335,8.90639), (5.18124,12.24596), (-5.01858,16.89203), (2.05561,12.69184), (-0.12117,15.59077), (0.99471,6.94287), (6.89979,-0.1801), (-4.18527,3.25318), (-6.35104,8.08804), (3.89734,13.78384), (-1.979,0.46434), (3.15404,7.78224), (3.52672,9.10987), (2.48372,-0.89391), (-6.13089,14.3696), (2.2968,3.01763), (-2.74324,8.03559), (-0.12876,7.24609), (-1.51135,11.86271), (-3.92434,6.28196), (-1.71254,8.9725), (-1.25878,14.46114), (2.03021,9.50216), (4.31726,16.30413), (-3.02908,1.02795), (9.7093,1.88717), (-3.36284,9.80106), (6.70938,4.53487), (0.42762,16.34543), (5.04726,7.71098), (2.78386,2.74639), (6.83022,6.51875), (-3.02109,10.42308), (-0.65382,13.57901), (-15.58675,0.52784), (5.89746,4.4708), (-4.11598,6.39619), (-1.37208,14.57666), (10.08082,2.71602), (5.35686,12.53905), (1.93331,11.4292), (10.47444,12.44641), (-2.36872,14.50894), (6.50752,17.64374), (2.54603,11.03218), (-0.4332,9.82789), (5.26572,10.11104), (2.09016,2.16137), (1.15513,10.24054), (14.95941,12.86909), (-3.85505,15.22845), (-2.36239,5.05411), (1.64338,10.84836), (-4.25074,11.15717), (7.29744,0.91782), (-1.18964,13.29961), (5.60612,15.11314), (-3.77011,11.54004), (6.67642,-0.94238), (-0.06862,19.32581), (5.60514,10.20744), (3.7341,6.54857), (9.59001,8.69108), (3.30093,8.2296), (-2.75658,8.4474), (4.71994,6.81178), (0.74699,5.99415), (2.91095,13.99336), (-7.36829,8.7469), (-5.29487,8.62349), (3.31079,1.84212), (1.06974,4.4762), (-1.18424,9.25421), (-7.415,10.44229), (3.40595,12.21649), (-7.63085,10.45968), (1.13336,15.34722), (-0.0096,5.50868), (0.8928,10.93609), (-0.5943,2.78631), (7.48306,11.86145), (10.11943,18.67385), (5.60459,10.64051), (4.00189,12.75565), (2.35823,6.63666), (0.33475,12.19343), (3.47072,9.08636), (-6.68867,11.67256), (3.31031,20.31392), (2.17159,11.66443); -SELECT '-28.740781574102936', '7.667329672103986e-133'; -SELECT roundBankers(studentTTest(left, right).1, 16) as t_stat, roundBankers(studentTTest(left, right).2, 16) as p_value from student_ttest; -DROP TABLE IF EXISTS student_ttest; diff --git a/tests/queries/0_stateless/01322_ttest_scipy.python b/tests/queries/0_stateless/01322_ttest_scipy.python deleted file mode 100644 index 66659e2ab71..00000000000 --- a/tests/queries/0_stateless/01322_ttest_scipy.python +++ /dev/null @@ -1,108 +0,0 @@ -#!/usr/bin/env python3 -import os -import io -import sys -import requests -import time -import pandas as pd -import numpy as np -from scipy import stats - -CLICKHOUSE_HOST = os.environ.get('CLICKHOUSE_HOST', '127.0.0.1') -CLICKHOUSE_PORT_HTTP = os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') -CLICKHOUSE_SERVER_URL_STR = 'http://' + ':'.join(str(s) for s in [CLICKHOUSE_HOST, CLICKHOUSE_PORT_HTTP]) + "/" - -class ClickHouseClient: - def __init__(self, host = CLICKHOUSE_SERVER_URL_STR): - self.host = host - - def query(self, query, connection_timeout = 1500): - NUMBER_OF_TRIES = 30 - DELAY = 10 - - for i in range(NUMBER_OF_TRIES): - r = requests.post( - self.host, - params = {'timeout_before_checking_execution_speed': 120, 'max_execution_time': 6000}, - timeout = connection_timeout, - data = query) - if r.status_code == 200: - return r.text - else: - print('ATTENTION: try #%d failed' % i) - if i != (NUMBER_OF_TRIES-1): - print(query) - print(r.text) - time.sleep(DELAY*(i+1)) - else: - raise ValueError(r.text) - - def query_return_df(self, query, connection_timeout = 1500): - data = self.query(query, connection_timeout) - df = pd.read_csv(io.StringIO(data), sep = '\t') - return df - - def query_with_data(self, query, content): - content = content.encode('utf-8') - r = requests.post(self.host, data=content) - result = r.text - if r.status_code == 200: - return result - else: - raise ValueError(r.text) - -def test_and_check(name, a, b, t_stat, p_value): - client = ClickHouseClient() - client.query("DROP TABLE IF EXISTS ttest;") - client.query("CREATE TABLE ttest (left Float64, right Float64) ENGINE = Memory;"); - client.query("INSERT INTO ttest VALUES {};".format(", ".join(['({},{})'.format(i, j) for i,j in zip(a, b)]))) - - real = client.query_return_df( - "SELECT roundBankers({}(left, right).1, 16) as t_stat, ".format(name) + - "roundBankers({}(left, right).2, 16) as p_value ".format(name) + - "FROM ttest FORMAT TabSeparatedWithNames;") - real_t_stat = real['t_stat'][0] - real_p_value = real['p_value'][0] - assert(abs(real_t_stat - np.float64(t_stat) < 1e-2)), "clickhouse_t_stat {}, scipy_t_stat {}".format(real_t_stat, t_stat) - assert(abs(real_p_value - np.float64(p_value)) < 1e-2), "clickhouse_p_value {}, scipy_p_value {}".format(real_p_value, p_value) - client.query("DROP TABLE IF EXISTS ttest;") - - -def test_student(): - rvs1 = np.round(stats.norm.rvs(loc=1, scale=5,size=500), 5) - rvs2 = np.round(stats.norm.rvs(loc=10, scale=5,size=500), 5) - s, p = stats.ttest_ind(rvs1, rvs2, equal_var = True) - test_and_check("studentTTest", rvs1, rvs2, s, p) - - rvs1 = np.round(stats.norm.rvs(loc=0, scale=5,size=500), 5) - rvs2 = np.round(stats.norm.rvs(loc=0, scale=5,size=500), 5) - s, p = stats.ttest_ind(rvs1, rvs2, equal_var = True) - test_and_check("studentTTest", rvs1, rvs2, s, p) - - - rvs1 = np.round(stats.norm.rvs(loc=0, scale=10,size=65536), 5) - rvs2 = np.round(stats.norm.rvs(loc=5, scale=1,size=65536), 5) - s, p = stats.ttest_ind(rvs1, rvs2, equal_var = True) - test_and_check("studentTTest", rvs1, rvs2, s, p) - -def test_welch(): - rvs1 = np.round(stats.norm.rvs(loc=1, scale=15,size=500), 5) - rvs2 = np.round(stats.norm.rvs(loc=10, scale=5,size=500), 5) - s, p = stats.ttest_ind(rvs1, rvs2, equal_var = True) - test_and_check("studentTTest", rvs1, rvs2, s, p) - - rvs1 = np.round(stats.norm.rvs(loc=0, scale=7,size=500), 5) - rvs2 = np.round(stats.norm.rvs(loc=0, scale=3,size=500), 5) - s, p = stats.ttest_ind(rvs1, rvs2, equal_var = True) - test_and_check("studentTTest", rvs1, rvs2, s, p) - - - rvs1 = np.round(stats.norm.rvs(loc=0, scale=10,size=65536), 5) - rvs2 = np.round(stats.norm.rvs(loc=5, scale=1,size=65536), 5) - s, p = stats.ttest_ind(rvs1, rvs2, equal_var = True) - test_and_check("studentTTest", rvs1, rvs2, s, p) - -if __name__ == "__main__": - test_student() - test_welch() - print("Ok.") \ No newline at end of file diff --git a/tests/queries/0_stateless/01322_ttest_scipy.sh b/tests/queries/0_stateless/01322_ttest_scipy.sh deleted file mode 100755 index 31c1acf3e60..00000000000 --- a/tests/queries/0_stateless/01322_ttest_scipy.sh +++ /dev/null @@ -1,8 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. "$CURDIR"/../shell_config.sh - -# We should have correct env vars from shell_config.sh to run this test - -python3 "$CURDIR"/01322_ttest_scipy.python diff --git a/tests/queries/0_stateless/01322_welch_ttest.sql b/tests/queries/0_stateless/01322_welch_ttest.sql deleted file mode 100644 index cce65c28bd8..00000000000 --- a/tests/queries/0_stateless/01322_welch_ttest.sql +++ /dev/null @@ -1,37 +0,0 @@ -/*Check only p-value first*/ -DROP TABLE IF EXISTS welch_ttest; -CREATE TABLE welch_ttest (left Float64, right Float64) ENGINE = Memory; -INSERT INTO welch_ttest VALUES (27.5,27.1), (21.0,22.0), (19.0,20.8), (23.6,23.4), (17.0,23.4), (17.9,23.5), (16.9,25.8), (20.1,22.0), (21.9,24.8), (22.6,20.2), (23.1,21.9), (19.6,22.1), (19.0,22.9), (21.7,20.5), (21.4,24.4); -SELECT '0.021378001462867'; -SELECT roundBankers(welchTTest(left, right).2, 16) from welch_ttest; -DROP TABLE IF EXISTS welch_ttest; - -CREATE TABLE welch_ttest (left Float64, right Float64) ENGINE = Memory; -INSERT INTO welch_ttest VALUES (30.02,29.89), (29.99,29.93), (30.11,29.72), (29.97,29.98), (30.01,30.02), (29.99,29.98); -SELECT '0.090773324285671'; -SELECT roundBankers(welchTTest(left, right).2, 16) from welch_ttest; -DROP TABLE IF EXISTS welch_ttest; - -CREATE TABLE welch_ttest (left Float64, right Float64) ENGINE = Memory; -INSERT INTO welch_ttest VALUES (0.010268,0.159258), (0.000167,0.136278), (0.000167,0.122389); -SELECT '0.00339907162713746'; -SELECT roundBankers(welchTTest(left, right).2, 16) from welch_ttest; -DROP TABLE IF EXISTS welch_ttest; - -/*Check t-stat and p-value and compare it with scipy.stat implementation - First: a=10, sigma (not sigma^2)=5, size=500 - Second: a=10, sigma = 10, size = 500 */ -CREATE TABLE welch_ttest (left Float64, right Float64) ENGINE = Memory; -INSERT INTO welch_ttest VALUES (14.72789,-8.65656), (9.61661,22.98234), (13.57615,23.80821), (3.98392,13.33939), (11.98889,-4.05537), (10.99422,23.5155), (5.44792,-6.45272), (20.29346,17.7903), (7.05926,11.463), (9.22732,5.28021), (12.06847,8.39157), (13.52612,6.02464), (8.24597,14.43732), (9.35245,15.76584), (10.12297,1.54391), (15.80624,1.24897), (13.68613,27.1507), (10.72729,7.71091), (5.62078,15.71846), (6.12229,32.97808), (6.03801,-1.79334), (8.95585,-9.23439), (24.04613,11.27838), (9.04757,0.72703), (2.68263,18.51557), (15.43935,9.16619), (2.89423,17.29624), (4.01423,-1.30208), (4.30568,-3.48018), (11.99948,10.12082), (8.40574,-8.01318), (10.86642,-14.22264), (9.4266,16.58174), (-8.12752,-0.55975), (7.91634,5.61449), (7.3967,1.44626), (2.26431,7.89158), (14.20118,1.13369), (6.68233,-0.82609), (15.46221,12.23365), (7.88467,12.45443), (11.20011,14.46915), (8.92027,13.72627), (10.27926,18.41459), (5.14395,29.66702), (5.62178,1.51619), (12.84383,10.40078), (9.98009,3.33266), (-0.69789,6.12036), (11.41386,11.86553), (7.76863,6.59422), (7.21743,22.0948), (1.81176,1.79623), (9.43762,14.29513), (19.22117,19.69162), (2.97128,-7.98033), (14.32851,5.48433), (7.54959,-2.28474), (3.81545,9.91876), (10.1281,10.64097), (2.48596,0.22523), (10.0461,17.01773), (3.59714,22.37388), (9.73522,14.04215), (18.8077,23.1244), (3.15148,18.96958), (12.26062,8.42663), (5.66707,3.7165), (6.58623,14.29366), (17.30902,23.50886), (9.91391,26.33722), (5.36946,26.72396), (15.73637,13.26287), (16.96281,12.97607), (11.54063,17.41838), (18.37358,8.63875), (11.38255,17.08943), (10.53256,23.15356), (8.08833,-4.4965), (16.27556,7.58895), (2.42969,26.04074), (9.56127,6.84245), (7.32998,20.56287), (9.19511,3.84735), (9.66903,-2.76304), (4.15029,13.1615), (8.83511,8.21954), (14.60617,-3.49943), (14.06143,22.12419), (5.39556,7.08323), (10.11871,16.12937), (10.56619,-0.32672), (14.4462,16.5942), (10.42106,7.68977), (7.75551,11.39484), (11.00418,-5.11987), (4.47226,20.87404), (16.35461,8.01007), (18.55174,3.26497), (11.82044,5.61253), (7.39454,20.69182), (11.27767,0.0296), (6.83827,21.904), (7.76858,22.46572), (15.97614,3.63685), (14.53781,-5.10846), (12.99546,14.86389), (16.91151,5.47188), (9.65012,18.44095), (14.25487,16.71368), (14.03618,6.36704), (2.57382,8.82663), (2.50779,14.6727), (14.24787,7.98383), (13.34666,2.65568), (7.31102,21.45827), (10.22981,11.77948), (17.4435,4.71979), (21.2074,3.17951), (6.64191,13.90226), (18.7086,15.50578), (14.78686,10.8026), (9.85287,16.91369), (4.48263,9.90552), (14.17469,13.87322), (14.4342,4.12366), (19.2481,-3.78985), (3.47165,1.7599), (8.28712,3.43715), (8.81657,-3.45246), (0.92319,23.64571), (20.41106,-4.96877), (6.76127,3.93514), (22.00242,1.49914), (8.66129,12.71519), (10.9929,5.11521), (17.95494,4.79872), (17.20996,20.89391), (12.18888,5.363), (12.14257,8.02765), (15.81243,14.30804), (4.43362,11.49002), (1.17567,14.25281), (15.60881,7.6573), (9.34833,15.49686), (6.33513,3.29327), (-0.83095,2.27236), (12.43268,12.58104), (6.63207,19.19128), (11.96877,15.25901), (14.81029,6.5221), (21.84876,10.10965), (3.75896,12.75249), (6.91307,16.50977), (13.73015,-8.6697), (8.63753,8.28553), (15.71679,1.44315), (1.74565,4.65869), (9.16895,0.98149), (5.70685,0.16623), (5.00117,17.66332), (13.06888,4.35346), (7.51204,6.52742), (15.34885,-1.06631), (5.20264,-5.28454), (8.59043,14.25583), (6.45619,8.74058), (14.61979,1.89553), (11.7075,-0.92959), (14.04901,10.30289), (4.20525,-6.3744), (15.1733,-8.1706), (3.12934,10.95369), (8.08049,4.94384), (15.41273,28.40568), (16.90751,3.7004), (5.86893,2.52363), (7.1086,4.07997), (4.418,7.8849), (12.0614,17.95409), (7.07887,16.67021), (3.61585,11.34377), (11.73001,-0.07446), (10.80449,22.00223), (8.40311,3.31778), (9.91276,18.50719), (16.4164,-3.58655), (5.25034,6.5394), (15.20283,12.40459), (10.42909,16.59866), (9.53888,7.54176), (14.68939,-1.51044), (6.60007,12.69758), (18.31058,2.9842), (7.01885,2.49187), (18.71631,2.04113), (10.50002,-2.46544), (10.7517,15.18368), (4.23224,-0.04058), (2.28924,-0.4127), (8.56059,10.5526), (8.25095,12.03982), (9.15673,12.10923), (13.28409,11.54954), (8.4513,-1.18613), (2.83911,11.30984), (2.79676,23.54105), (9.11055,10.67321), (7.18529,24.09196), (-4.1258,7.5008), (5.28306,12.52233), (6.82757,4.30673), (10.89035,9.35793), (5.24822,4.44472), (11.935,-7.00679), (6.45675,8.56241), (10.18088,23.73891), (4.9932,15.62708), (18.09939,16.09205), (8.11738,12.52074), (5.37883,14.58927), (10.50339,-4.80187), (16.64093,8.47964), (14.77263,7.75477), (13.71385,12.6893), (6.98746,7.14147), (10.74635,12.12654), (5.49432,12.32334), (13.46078,7.98909), (10.67565,3.26652), (9.0291,20.53684), (11.51417,32.3369), (13.07118,19.74911), (9.5049,-4.62897), (8.50611,8.26483), (6.47606,20.88451), (13.06526,-2.12982), (19.08658,25.61459), (9.49741,5.32091), (10.60865,-4.1196), (2.28996,7.57937), (8.12846,21.15847), (5.62241,6.46355), (4.07712,7.74846), (17.98526,19.62636), (9.466,28.34629), (11.38904,26.73919), (5.91826,20.40427), (1.52059,3.03378), (18.79161,10.2537), (18.20669,7.47745), (-1.67829,10.79184), (18.01586,3.91962), (16.31577,19.97973), (7.88281,18.87711), (8.46179,12.56157), (10.31113,11.46033), (14.88377,3.78661), (1.31835,-9.45748), (2.53176,12.06033), (9.48625,-0.74615), (3.97936,13.2815), (11.52319,24.78052), (13.24178,5.83337), (7.58739,17.4111), (10.00959,19.70331), (9.73361,11.78446), (8.35716,-1.366), (1.65491,1.37458), (11.11521,16.31483), (6.08355,32.63464), (10.04582,-3.79736), (11.58237,19.17984), (16.40249,-0.27705), (1.9691,-3.69456), (13.22776,28.38058), (2.67059,-1.36876), (9.83651,-25.63301), (2.12539,3.58644), (9.27114,-6.85667), (9.0699,13.42225), (2.78179,12.04671), (12.49311,28.99468), (12.97662,7.87662), (15.06359,2.61119), (16.91565,-3.56022), (5.92011,1.50022), (5.81304,14.55836), (8.46425,9.35831), (9.48705,16.9366), (4.68191,29.23126), (5.70028,15.31386), (-0.78798,13.46112), (10.03442,7.39667), (15.45433,11.15599), (9.43845,9.80499), (3.05825,22.64923), (6.92126,8.67693), (14.05905,18.67335), (19.71579,-3.19127), (15.0131,22.94716), (4.50386,17.86834), (1.31061,16.98267), (10.81197,15.91653), (14.32942,11.79718), (9.26469,18.50208), (7.27679,8.90755), (22.69295,10.44843), (12.03763,4.67433), (7.34876,6.82287), (16.60689,10.82228), (7.48786,-4.18631), (15.78602,20.3872), (17.21048,11.84735), (13.93482,21.25376), (9.69911,10.55032), (12.24315,12.19023), (10.58131,0.63369), (19.57006,7.92381), (9.8856,17.90933), (11.70302,15.30781), (7.89864,10.01877), (12.24831,0.88744), (16.93707,22.20967), (9.65467,-4.23117), (4.221,21.50819), (15.45229,11.27421), (12.83088,-16.23179), (7.58313,33.43085), (12.895,5.15093), (10.02471,1.34505), (13.36059,6.027), (5.07864,-10.43035), (9.72017,27.45998), (11.05809,19.24886), (15.28528,-4.44761), (13.99834,5.453), (19.26989,12.73758), (9.41846,11.2897), (11.65425,31.032), (8.49638,7.39168), (6.38592,11.95245), (-4.69837,26.279), (12.22061,-1.0255), (9.41331,10.36675), (13.2075,11.58439), (12.97005,27.8405), (11.44352,13.1707), (9.79805,31.39133), (6.93116,27.08301), (10.07691,-2.14368), (22.05892,4.08476), (7.80353,21.5573), (-2.17276,16.69822), (0.61509,7.69955), (8.35842,8.32793), (17.77108,6.49235), (14.70841,-7.3284), (1.27992,10.58264), (15.62699,-6.17006), (9.32914,34.55782), (15.41866,10.93221), (10.82009,44.24299), (3.29902,14.6224), (9.21998,-7.42798), (7.93845,15.52351), (10.33344,11.33982), (12.06399,10.46716), (5.5308,13.0986), (8.38727,-4.25988), (18.11104,9.55316), (8.86565,0.75489), (19.41825,25.99212), (9.52376,-0.81401), (3.94552,3.49551), (9.37587,22.99402), (15.44954,10.99628), (15.90527,23.70223), (13.18927,2.71482), (7.01646,22.82309), (9.06005,31.25686), (9.06431,4.86318), (5.76006,-1.06476), (9.18705,15.10298), (-3.48446,-0.61015), (15.89817,17.81246), (12.94719,-1.55788), (23.69426,18.09709), (17.47755,9.11271), (15.61528,9.94682), (0.54832,-7.33194), (14.32916,-4.67293), (9.55305,21.81717), (13.79891,7.16318), (0.82544,13.25649), (13.34875,13.88776), (9.07614,4.95793), (5.19621,17.65303), (2.1451,14.47382), (9.87726,13.19373), (8.45439,31.86093), (-1.41842,5.73161), (7.93598,10.96492), (11.23151,6.97951), (17.84458,1.75136), (7.02237,10.96144), (10.7842,15.08137), (4.42832,9.95311), (4.45044,7.07729), (1.50938,3.08148), (21.21651,22.37954), (6.2097,8.51951), (6.84354,2.88746), (18.53804,26.73509), (12.01072,-2.88939), (4.8345,-2.82367), (20.41587,-0.35783), (14.48353,14.22076), (8.71116,11.50295), (12.42818,7.10171), (14.89244,8.28488), (8.03033,0.54178), (5.25917,13.8022), (2.30092,15.62157), (10.22504,10.79173), (15.37573,28.18946), (7.13666,30.43524), (4.45018,2.54914), (10.18405,9.89421), (3.91025,13.08631), (14.52304,4.68761), (13.14771,5.61516), (11.99219,22.88072), (9.21345,7.4735), (8.85106,11.27382), (12.91887,2.39559), (15.62308,-3.31889), (11.88034,9.61957), (15.12097,23.01381), (11.58168,-1.23467), (16.83051,9.07691), (5.25405,15.78056), (2.19976,12.28421), (4.56716,9.44888), (16.46053,13.16928), (5.61995,4.33357), (8.67704,2.21737), (5.62789,33.17833), (9.84815,13.25407), (13.05834,-2.47961), (11.74205,6.41401), (3.88393,18.8439), (16.15321,-4.63375), (4.83925,-8.2909), (13.00334,12.18221), (4.4028,-2.95356), (4.35794,19.61659), (4.47478,12.45056), (2.38713,-4.17198), (4.25235,21.9641), (10.87509,11.96416), (9.82411,12.74573), (13.61518,10.47873), (10.25507,12.73295), (4.0335,11.31373), (10.69881,9.9827), (5.70321,5.87138), (6.96244,4.24372), (9.35874,-23.72256), (6.28076,28.41337), (8.29015,4.88103), (6.88653,3.61902), (7.70687,8.93586), (8.2001,16.40759), (6.73415,27.84494), (3.82052,5.6001), (3.94469,14.51379), (15.82384,13.5576), (2.54004,12.92213), (10.74876,3.90686), (12.60517,17.07104), (17.7024,15.84268), (4.6722,17.38777), (13.67341,16.54766), (6.4565,5.94487), (12.95699,17.02804), (4.56912,7.66386), (5.58464,10.43088), (4.0638,6.16059), (13.05559,20.46178), (5.38269,20.02888), (0.16354,20.95949), (7.23962,6.50808), (7.38577,7.22366), (8.50951,8.06659), (13.72574,16.08241), (17.80421,13.83514), (3.01135,-0.33454), (8.02608,12.98848), (14.23847,12.99024); -SELECT '-0.5028215369186904', '0.6152361677168877'; -SELECT roundBankers(welchTTest(left, right).1, 16) as t_stat, roundBankers(welchTTest(left, right).2, 16) as p_value from welch_ttest; -DROP TABLE IF EXISTS welch_ttest; - -/*Check t-stat and p-value and compare it with scipy.stat implementation - First: a=10, sigma (not sigma^2)=5, size=500 - Second: a=1, sigma = 12, size = 500 */ -CREATE TABLE welch_ttest (left Float64, right Float64) ENGINE = Memory; -INSERT INTO welch_ttest VALUES (4.82025,-2.69857), (6.13896,15.80943), (15.20277,7.31555), (14.15351,3.96517), (7.21338,4.77809), (8.55506,9.6472), (13.80816,-26.41717), (11.28411,-10.85635), (7.4612,-1.4376), (7.43759,-0.96308), (12.9832,2.84315), (-5.74783,5.79467), (12.47114,-3.06091), (15.14223,-14.62902), (3.40603,22.08022), (9.27323,-2.11982), (7.88547,-4.84824), (8.56456,-10.50447), (4.59731,2.4891), (7.91213,9.90324), (7.33894,-22.66866), (21.74811,-0.97103), (11.92111,-16.57608), (0.18828,-3.78749), (10.47314,25.84511), (20.37396,5.30797), (11.04991,-18.19466), (13.30083,11.72708), (14.28065,0.2891), (2.86942,-9.83474), (24.96072,6.69942), (14.20164,18.09604), (18.28769,18.52651), (10.50949,1.38201), (9.22273,7.64615), (11.77608,17.66598), (8.56872,-2.44141), (13.74535,-9.01598), (11.65209,27.69142), (12.51894,4.06946), (17.76256,-15.0077), (13.52122,-10.49648), (8.70796,-4.88322), (6.04749,-25.09805), (16.33064,-4.64024), (8.35636,20.94434), (14.03496,24.12126), (11.05834,-14.10962), (14.49261,10.6512), (2.59383,14.50687), (8.01022,-19.88081), (4.05458,-11.55271), (13.26384,13.16921), (14.62058,16.63864), (10.52489,-24.08114), (8.46357,-9.09949), (6.4147,-10.54702), (9.70071,0.20813), (12.47581,8.19066), (4.38333,-2.70523), (17.54172,-0.23954), (10.12109,7.19398), (7.73186,-7.1618), (14.0279,-7.44322), (11.6621,-17.92031), (17.47045,-1.58146), (15.50223,9.18338), (15.46034,3.25838), (13.39964,-14.30234), (14.98025,1.84695), (15.87912,31.13794), (17.67374,-0.85067), (9.64073,19.02787), (12.84904,-3.09594), (7.70278,13.45584), (13.03156,-5.48104), (9.04512,-22.74928), (15.97014,-8.03697), (8.96389,17.31143), (11.48009,-16.65231), (9.71153,-18.58713), (13.00084,-16.52641), (12.39803,14.95261), (13.08188,12.56762), (5.82244,15.00188), (10.81871,1.85858), (8.2539,2.1926), (7.52114,-2.4095), (9.11488,21.56873), (8.37482,3.35509), (14.48652,-4.98672), (11.42152,35.08603), (16.03111,-10.01602), (13.14057,-3.85153), (-2.26351,-6.81974), (15.50394,19.56525), (14.88603,-9.35488), (13.37257,0.24268), (11.84026,-3.51488), (7.66558,-0.37066), (6.24584,24.20888), (3.6312,-11.73537), (2.7018,0.01282), (5.63656,0.03963), (5.82643,-9.65589), (10.06745,-0.37429), (-0.5831,5.61255), (14.84202,0.49984), (9.5524,-10.15066), (19.71713,-14.54314), (14.23109,16.56889), (8.69105,-7.73873), (5.33742,-3.76422), (7.30372,1.40722), (7.93342,2.28818), (15.20884,-13.12643), (7.53839,5.17082), (13.45311,4.79089), (11.04473,-17.42643), (10.76673,8.72548), (15.44145,-3.70285), (14.06596,16.77893), (9.14873,13.382), (12.88372,19.98418), (8.74994,0.00483), (10.53263,-4.75951), (16.16694,2.35391), (8.37197,21.65809), (3.43739,-9.2714), (4.72799,-18.38253), (9.08802,7.23097), (11.2531,14.97927), (5.16115,-4.02197), (10.20895,-29.8189), (18.70884,-12.8554), (15.88924,-7.60124), (3.38758,-14.90158), (6.46449,-3.31486), (10.21088,31.38144), (14.08458,-8.61288), (15.74508,15.31895), (19.31896,-10.19488), (13.19641,13.796), (11.95409,-0.32912), (10.70718,-0.0684), (1.05245,-30.06834), (10.04772,24.93912), (17.01369,-3.26506), (10.2286,-8.29751), (19.58323,-5.39189), (7.02892,-25.08603), (4.16866,-1.45318), (8.94326,16.72724), (4.99854,-3.38467), (8.88352,-26.00478), (18.65422,7.28369), (17.32328,16.96226), (9.33492,16.5858), (14.94788,10.46583), (8.05863,3.84345), (14.6737,-2.99382), (10.93801,1.42078), (0.54036,-11.0123), (-0.34242,2.09909), (5.89076,1.21064), (3.15189,15.36079), (1.94421,-21.61349), (6.38698,22.7726), (10.50654,10.50512), (8.95362,-6.95825), (6.23711,9.20036), (11.75359,15.66902), (12.42155,3.28098), (-1.55472,-9.05692), (4.6688,0.32882), (10.48087,-1.64934), (11.74615,-4.81406), (9.26822,-5.06006), (7.55517,19.97493), (12.76005,2.88646), (16.47102,-0.34552), (11.31297,7.55186), (14.37437,-22.96115), (2.38799,31.29166), (6.44577,6.18798), (5.07471,-2.52715), (11.55123,-11.58799), (7.76795,14.13596), (10.60116,13.45069), (14.40885,12.15179), (11.58158,3.44491), (8.81648,-8.78006), (12.92299,18.32087), (11.26939,11.91757), (17.95014,-2.00179), (2.95002,10.88411), (17.41959,9.09327), (11.12455,6.62484), (8.78541,8.87178), (14.36413,11.52254), (12.98554,-14.15988), (12.58505,-17.19515), (15.49789,14.03089), (11.70999,-2.4095), (0.65596,-16.83575), (11.08202,2.71469), (14.75752,4.84351), (6.84385,-1.17651), (9.27245,-3.37529), (13.78243,-19.92137), (17.4863,4.48952), (4.01777,-12.4906), (11.82861,-5.65277), (13.86551,8.50819), (6.16591,-19.61261), (8.71589,12.54156), (16.77195,11.06784), (17.23243,-12.59285), (-2.12941,3.43683), (5.66629,-3.00325), (12.45153,12.49082), (1.63971,7.20955), (13.84031,17.6547), (4.6144,15.8619), (5.26169,24.3048), (9.27769,-8.05434), (9.14288,-6.06901), (9.71953,-15.69515), (9.38446,-11.13917), (1.64788,-3.90757), (11.72922,-2.57038), (13.68926,5.14065), (9.42952,17.8497), (12.05574,-8.64665), (9.09148,-18.68331), (5.32273,5.8567), (20.25258,-20.93884), (10.14599,4.40583), (10.82156,14.35985), (5.75736,4.18134), (7.13567,4.3635), (9.29746,9.35428), (5.1618,2.8908), (10.076,16.01017), (21.65669,-1.48499), (13.35486,-9.97949), (6.79957,1.03055), (8.76243,-2.79697), (14.59294,6.85977), (16.90609,4.73213), (10.50337,2.7815), (-0.07923,-2.46866), (13.51648,18.39425), (12.0676,-0.80378), (0.86482,-0.22982), (9.03563,-16.11608), (5.38751,3.0862), (17.16866,3.20779), (2.78702,10.50146), (11.15548,-0.21305), (12.30843,11.21012), (8.04897,-0.99825), (9.95814,18.39633), (11.29308,-3.39003), (14.13032,-0.64411), (21.05877,-1.39932), (3.57386,15.45319), (7.96631,-0.66044), (3.30484,-15.2223), (18.61856,-34.39907), (16.35184,-3.57836), (7.65236,16.82828), (18.02895,1.66624), (9.79458,15.43475), (16.7274,8.17776), (8.84453,5.50486), (13.05709,10.43082), (10.91447,-6.63332), (8.40171,2.28008), (16.95211,16.37203), (11.82194,5.16313), (19.87978,-8.85281), (12.88455,13.26692), (-0.00947,-7.46842), (12.28109,8.43091), (6.96462,-13.18172), (13.75282,-0.72401), (14.39141,22.3881), (11.07193,10.65448), (12.88039,2.81289), (11.38253,10.92405), (21.02707,-8.95358), (7.51955,19.80653), (6.31984,-12.86527), (15.6543,5.38826), (14.80315,-6.83501), (8.38024,-15.7647), (21.7516,-27.67412), (14.31336,8.6499), (15.04703,-4.89542), (5.73787,16.76167), (13.16911,12.84284), (12.40695,-17.27324), (9.88968,-4.18726), (8.46703,-14.62366), (8.70637,-5.49863), (8.03551,-16.22846), (5.9757,10.60329), (12.22951,6.46781), (3.14736,1.70458), (10.51266,10.77448), (18.593,0.8463), (10.82213,13.0482), (7.14216,-4.36264), (6.81154,3.22647), (-0.6486,2.38828), (20.56136,6.7946), (11.35367,-0.25254), (11.38205,1.2497), (17.14,1.6544), (14.91215,4.1019), (15.50207,11.27839), (5.93162,-5.04127), (3.74869,18.11674), (14.11532,0.51231), (7.38954,-0.51029), (5.45764,13.52556), (18.33733,16.10171), (9.91923,5.68197), (2.38991,-2.85904), (14.16756,-8.89167), (2.39791,6.24489), (6.92586,10.85319), (5.32474,-0.39816), (2.28812,3.87079), (5.71718,-3.1867), (5.84197,1.55322), (2.76206,16.86779), (19.05928,-14.60321), (11.51788,-1.81952), (6.56648,-3.11624), (3.35735,1.24193), (7.55948,10.18179), (19.99908,4.69796), (13.00634,0.69032), (18.36886,11.7723), (11.14675,7.62896), (16.72931,9.89741), (12.50106,9.11484), (6.00605,-3.84676), (23.06653,-0.4777), (5.39694,0.95958), (9.53167,-7.95056), (12.76944,-10.97474), (7.20604,-6.54861), (13.25391,34.74933), (13.7341,27.39463), (10.85292,4.18299), (-7.75835,6.02476), (10.29728,-1.99397), (13.70099,1.26478), (10.17959,23.37106), (9.98399,10.49682), (12.69389,-11.04354), (-0.28848,-12.22284), (-2.18319,-9.87635), (13.36378,28.90511), (10.09232,6.77613), (5.49489,0.55352), (5.46156,0.37031), (0.94225,7.1418), (12.79205,3.24897), (10.09593,-1.60918), (6.06218,3.1675), (0.89463,-17.97072), (11.88986,-5.61743), (10.79733,14.1422), (1.51371,14.87695), (2.20967,-4.65961), (15.45732,-0.99174), (16.5262,-2.96623), (5.99724,-9.02263), (8.3613,-17.2088), (15.68183,2.78608), (15.32117,6.74239), (14.15674,4.8524), (6.64553,7.46731), (4.20777,1.04894), (-0.10521,-12.8023), (-0.88169,-17.18188), (1.85913,-5.08801), (9.73673,22.13942), (0.30926,-0.36384), (6.17559,17.80564), (11.76602,7.67504), (5.68385,1.59779), (14.57088,4.10942), (12.81509,0.61074), (9.85682,-14.40767), (12.06376,10.59906), (6.08874,16.57017), (11.63921,-15.17526), (14.86722,-6.98549), (10.41035,-0.64548), (2.93794,3.23756), (12.21841,14.65504), (0.23804,4.583), (3.14845,12.72378), (7.29748,5.26547), (3.06134,0.81781), (13.77684,9.38273), (16.21992,10.37636), (5.33511,10.70325), (9.68959,-0.83043), (9.44169,-7.53149), (18.08012,-9.09147), (4.04224,-19.51381), (8.77918,-28.44508), (10.18324,6.44392), (9.38914,11.10201), (11.76995,-2.86184), (14.19963,8.30673), (6.88817,8.8797), (16.56123,10.68053), (15.39885,15.62919), (5.21241,8.00579), (4.44408,6.4651), (17.87587,-4.50029), (12.53337,18.04514), (13.60916,11.12996), (6.60104,-5.14007), (7.35453,9.43857), (18.61572,3.13476), (6.10437,4.9772), (13.08682,-17.45782), (12.15404,0.05552), (4.90789,-1.90283), (2.13353,2.67908), (12.49593,-2.62243), (11.93056,-3.22767), (13.29408,-8.70222), (5.70038,-23.11605), (8.40271,21.6757), (5.19456,12.70076), (-5.51028,4.4322), (14.0329,11.69344), (10.38365,9.18052), (6.56812,-2.2549), (4.21129,-2.15615), (9.7157,20.29765), (9.88553,-0.29536), (13.45346,15.50109), (4.97752,8.79187), (12.77595,5.11533), (8.56465,-20.44436), (4.27703,-3.00909), (18.12502,-4.48291), (12.45735,21.84462), (12.42912,1.94225), (12.08125,-2.81908), (10.85779,17.19418), (4.36013,-9.33528), (11.85062,-0.17346), (8.47776,0.03958), (9.60822,-35.17786), (11.3069,8.36887), (14.25525,-9.02292), (1.55168,-10.98804), (14.57782,0.29335), (7.84786,4.29634), (9.87774,3.87718), (14.75575,-9.08532), (3.68774,7.13922), (9.37667,-7.62463), (20.28676,-10.5666), (12.10027,4.68165), (8.01819,-3.30172), (18.78158,13.04852), (20.85402,13.45616), (18.98069,2.41043), (16.1429,-0.36501), (9.24047,-15.67383), (14.12487,17.92217), (10.18841,8.42106), (-3.04478,3.22063), (5.7552,-7.31753), (9.30376,21.99596), (11.42837,-36.8273), (6.02364,-20.46391), (8.86984,5.74179), (10.91177,-15.83178), (10.04418,14.90454), (18.10774,-8.84645), (7.49384,3.72036), (9.11556,4.6877), (9.7051,16.35418), (5.23268,3.15441), (9.04647,2.39907), (8.81547,-17.58664), (2.65098,-13.18269); -SELECT '14.971190998235835', '5.898143508382202e-44'; -SELECT roundBankers(welchTTest(left, right).1, 16) as t_stat, roundBankers(welchTTest(left, right).2, 16) as p_value from welch_ttest; -DROP TABLE IF EXISTS welch_ttest; From ca0037d18b73956301b6bebec4ef364921944e2a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 4 Nov 2020 18:07:18 +0300 Subject: [PATCH 202/314] Update group_by_sundy_li.xml --- tests/performance/group_by_sundy_li.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/group_by_sundy_li.xml b/tests/performance/group_by_sundy_li.xml index 4c9e397520a..3fcc4acf88d 100644 --- a/tests/performance/group_by_sundy_li.xml +++ b/tests/performance/group_by_sundy_li.xml @@ -1,4 +1,4 @@ - + CREATE TABLE a ( From 8d5fe1148c721a3065cbebcea5f02ae9689876f4 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 4 Nov 2020 18:07:34 +0300 Subject: [PATCH 203/314] add client config --- docker/test/performance-comparison/compare.sh | 2 +- .../config/client_config.xml | 17 +++++++++++++++++ 2 files changed, 18 insertions(+), 1 deletion(-) create mode 100644 docker/test/performance-comparison/config/client_config.xml diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 08462298093..60d226f025d 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1092,7 +1092,7 @@ function upload_results --secure \ --user "${CHPC_DATABASE_USER}" \ --password "${CHPC_DATABASE_PASSWORD}" \ - --config "ch/tests/config/client_config.xml" \ + --config "right/config/client_config.xml" \ --database perftest \ --date_time_input_format=best_effort \ --query " diff --git a/docker/test/performance-comparison/config/client_config.xml b/docker/test/performance-comparison/config/client_config.xml new file mode 100644 index 00000000000..9f590389dc3 --- /dev/null +++ b/docker/test/performance-comparison/config/client_config.xml @@ -0,0 +1,17 @@ + + + + + true + true + sslv2,sslv3 + true + + AcceptCertificateHandler + + + + From c496aaddb286bcac40aa649fbe2d1e8e9f0803d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 18:29:43 +0300 Subject: [PATCH 204/314] Add a test for #16231 --- tests/queries/0_stateless/01556_if_null.reference | 1 + tests/queries/0_stateless/01556_if_null.sql | 5 +++++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/01556_if_null.reference create mode 100644 tests/queries/0_stateless/01556_if_null.sql diff --git a/tests/queries/0_stateless/01556_if_null.reference b/tests/queries/0_stateless/01556_if_null.reference new file mode 100644 index 00000000000..a0c5e7faf40 --- /dev/null +++ b/tests/queries/0_stateless/01556_if_null.reference @@ -0,0 +1 @@ +([1],[5]) 4 4 diff --git a/tests/queries/0_stateless/01556_if_null.sql b/tests/queries/0_stateless/01556_if_null.sql new file mode 100644 index 00000000000..1952b4b7617 --- /dev/null +++ b/tests/queries/0_stateless/01556_if_null.sql @@ -0,0 +1,5 @@ +SELECT + sumMapIf([1], [1], nullIf(number, 3) > 0) AS col1, + countIf(1, nullIf(number, 3) > 0) AS col2, + sumIf(1, nullIf(number, 3) > 0) AS col3 +FROM numbers(1, 5); From 1bcf22d42fc211350f2ba830a508f67966fd6f93 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 18:59:14 +0300 Subject: [PATCH 205/314] Fix 'max_parallel_replicas' without sampling. --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index f06bfb97b2c..2ca989e12e6 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -366,6 +366,15 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( * It is also important that the entire universe can be covered using SAMPLE 0.1 OFFSET 0, ... OFFSET 0.9 and similar decimals. */ + /// Parallel replicas has been requested but there is no way to sample data. + /// Select all data from first replica and no data from other replicas. + if (settings.parallel_replicas_count > 1 && !data.supportsSampling() && settings.parallel_replica_offset > 0) + { + LOG_DEBUG(log, "Will use no data on this replica because parallel replicas processing has been requested" + " (the setting 'max_parallel_replicas') but the table does not support sampling and this replica is not the first."); + return {}; + } + bool use_sampling = relative_sample_size > 0 || (settings.parallel_replicas_count > 1 && data.supportsSampling()); bool no_data = false; /// There is nothing left after sampling. From a4cfc240e3a76166dbde8f4cc9b80279af35c97d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 19:19:59 +0300 Subject: [PATCH 206/314] Add a test --- ..._max_parallel_replicas_no_sample.reference | 4 ++++ .../01557_max_parallel_replicas_no_sample.sql | 22 +++++++++++++++++++ .../queries/0_stateless/arcadia_skip_list.txt | 1 + 3 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.reference create mode 100644 tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql diff --git a/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.reference b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.reference new file mode 100644 index 00000000000..2715babfff2 --- /dev/null +++ b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.reference @@ -0,0 +1,4 @@ +Hello +1000 +1000 +1000 diff --git a/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql new file mode 100644 index 00000000000..d86f692a1ea --- /dev/null +++ b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS t; +CREATE TABLE t (x String) ENGINE = MergeTree ORDER BY x; +INSERT INTO t VALUES ('Hello'); + +SET max_parallel_replicas = 3; +SELECT * FROM remote('127.0.0.{2|3|4}', currentDatabase(), t); + +DROP TABLE t; + +CREATE TABLE t (x String) ENGINE = MergeTree ORDER BY cityHash64(x) SAMPLE BY cityHash64(x); +INSERT INTO t SELECT toString(number) FROM numbers(1000); + +SET max_parallel_replicas = 1; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t); + +SET max_parallel_replicas = 2; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t); + +SET max_parallel_replicas = 3; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t); + +DROP TABLE t; diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 8ece63c419c..900cc82b33f 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -159,3 +159,4 @@ 01547_query_log_current_database 01548_query_log_query_execution_ms 01552_dict_fixedstring +01557_max_parallel_replicas_no_sample.sql From 1c961c281cd7365a977e36f7360652fb4fcff101 Mon Sep 17 00:00:00 2001 From: Jacob Hayes Date: Wed, 4 Nov 2020 11:32:58 -0500 Subject: [PATCH 207/314] Add farmFingerprint64 test --- tests/queries/0_stateless/00751_hashing_ints.reference | 2 ++ tests/queries/0_stateless/00751_hashing_ints.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/00751_hashing_ints.reference b/tests/queries/0_stateless/00751_hashing_ints.reference index 8eccccd0449..31acc13a762 100644 --- a/tests/queries/0_stateless/00751_hashing_ints.reference +++ b/tests/queries/0_stateless/00751_hashing_ints.reference @@ -2,6 +2,8 @@ 11717965186011240346 13379111408315310133 13379111408315310133 +7773179648686038998 +13379111408315310133 13379111408315310133 623211862 9052087431341907723 diff --git a/tests/queries/0_stateless/00751_hashing_ints.sql b/tests/queries/0_stateless/00751_hashing_ints.sql index d2f0a26cef4..be3268da29d 100644 --- a/tests/queries/0_stateless/00751_hashing_ints.sql +++ b/tests/queries/0_stateless/00751_hashing_ints.sql @@ -1,6 +1,8 @@ SELECT halfMD5(123456); SELECT sipHash64(123456); SELECT cityHash64(123456); +SELECT farmFingerprint64(123456); +SELECT farmFingerprint64('123456'); SELECT farmHash64(123456); SELECT metroHash64(123456); SELECT murmurHash2_32(123456); From 22954cc12393f8a0ad9c345eec094cacd8d01fa8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 19:48:36 +0300 Subject: [PATCH 208/314] Fix UBSan report when trying to convert infinite floating point number to integer --- src/Common/FieldVisitors.h | 5 +++++ .../01557_field_infinite_convert_to_number.reference | 0 .../0_stateless/01557_field_infinite_convert_to_number.sql | 1 + 3 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/01557_field_infinite_convert_to_number.reference create mode 100644 tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql diff --git a/src/Common/FieldVisitors.h b/src/Common/FieldVisitors.h index 4fcb3091833..421423131a2 100644 --- a/src/Common/FieldVisitors.h +++ b/src/Common/FieldVisitors.h @@ -3,6 +3,7 @@ #include #include #include +#include class SipHash; @@ -142,6 +143,10 @@ public: T operator() (const Float64 & x) const { + if constexpr (!std::is_floating_point_v) + if (!isFinite(x)) + throw Exception("Cannot convert infinite value to integer type", ErrorCodes::CANNOT_CONVERT_TYPE); + if constexpr (std::is_same_v) return Int256(x); else diff --git a/tests/queries/0_stateless/01557_field_infinite_convert_to_number.reference b/tests/queries/0_stateless/01557_field_infinite_convert_to_number.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql b/tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql new file mode 100644 index 00000000000..edc4d5cbc91 --- /dev/null +++ b/tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql @@ -0,0 +1 @@ +SET max_threads = nan; -- { serverError 70 } From 97b4534eb3a4ca9f8aad99dc25c4e73f7f66cb2a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 4 Nov 2020 19:57:00 +0300 Subject: [PATCH 209/314] Update AggregateFunctionOrFill.cpp --- src/AggregateFunctions/AggregateFunctionOrFill.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionOrFill.cpp b/src/AggregateFunctions/AggregateFunctionOrFill.cpp index 959836c0e49..ee9fce710e6 100644 --- a/src/AggregateFunctions/AggregateFunctionOrFill.cpp +++ b/src/AggregateFunctions/AggregateFunctionOrFill.cpp @@ -21,7 +21,7 @@ private: Kind kind; public: - AggregateFunctionCombinatorOrFill(Kind kind_) : kind(kind_) {} + explicit AggregateFunctionCombinatorOrFill(Kind kind_) : kind(kind_) {} String getName() const override { From fa72fed4ad10cf6a71ef940fde3db4add709d041 Mon Sep 17 00:00:00 2001 From: damozhaeva <68770561+damozhaeva@users.noreply.github.com> Date: Wed, 4 Nov 2020 22:17:02 +0300 Subject: [PATCH 210/314] DOCSUP-3173 Edit and translate to Russian (#16462) * Edit and translate to Russian. * v2 27.10 * other-functions * Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: BayoNet * Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: BayoNet Co-authored-by: Daria Mozhaeva Co-authored-by: BayoNet --- .../reference/quantileexact.md | 13 +- .../aggregate-functions/reference/index.md | 138 +++++------ .../reference/quantileexact.md | 222 +++++++++++++----- .../functions/other-functions.md | 42 ++++ 4 files changed, 285 insertions(+), 130 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantileexact.md b/docs/en/sql-reference/aggregate-functions/reference/quantileexact.md index 40b25c14988..a39f724f368 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantileexact.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantileexact.md @@ -53,13 +53,13 @@ Result: Similar to `quantileExact`, this computes the exact [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence. -To get exact value, all the passed values are combined into an array, which is then fully sorted. The sorting [algorithm's](https://en.cppreference.com/w/cpp/algorithm/sort) complexity is `O(N·log(N))`, where `N = std::distance(first, last)` comparisons. +To get the exact value, all the passed values are combined into an array, which is then fully sorted. The sorting [algorithm's](https://en.cppreference.com/w/cpp/algorithm/sort) complexity is `O(N·log(N))`, where `N = std::distance(first, last)` comparisons. -Depending on the level, i.e if the level is 0.5 then the exact lower median value is returned if there are even number of elements and the middle value is returned if there are odd number of elements. Median is calculated similar to the [median_low](https://docs.python.org/3/library/statistics.html#statistics.median_low) implementation which is used in python. +The return value depends on the quantile level and the number of elements in the selection, i.e. if the level is 0.5, then the function returns the lower median value for an even number of elements and the middle median value for an odd number of elements. Median is calculated similarly to the [median_low](https://docs.python.org/3/library/statistics.html#statistics.median_low) implementation which is used in python. -For all other levels, the element at the the index corresponding to the value of `level * size_of_array` is returned. For example: +For all other levels, the element at the index corresponding to the value of `level * size_of_array` is returned. For example: -```$sql +``` sql SELECT quantileExactLow(0.1)(number) FROM numbers(10) ┌─quantileExactLow(0.1)(number)─┐ @@ -111,9 +111,10 @@ Result: Similar to `quantileExact`, this computes the exact [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence. -To get exact value, all the passed values are combined into an array, which is then fully sorted. The sorting [algorithm's](https://en.cppreference.com/w/cpp/algorithm/sort) complexity is `O(N·log(N))`, where `N = std::distance(first, last)` comparisons. +All the passed values are combined into an array, which is then fully sorted, +to get the exact value. The sorting [algorithm's](https://en.cppreference.com/w/cpp/algorithm/sort) complexity is `O(N·log(N))`, where `N = std::distance(first, last)` comparisons. -Depending on the level, i.e if the level is 0.5 then the exact higher median value is returned if there are even number of elements and the middle value is returned if there are odd number of elements. Median is calculated similar to the [median_high](https://docs.python.org/3/library/statistics.html#statistics.median_high) implementation which is used in python. For all other levels, the element at the the index corresponding to the value of `level * size_of_array` is returned. +The return value depends on the quantile level and the number of elements in the selection, i.e. if the level is 0.5, then the function returns the higher median value for an even number of elements and the middle median value for an odd number of elements. Median is calculated similarly to the [median_high](https://docs.python.org/3/library/statistics.html#statistics.median_high) implementation which is used in python. For all other levels, the element at the index corresponding to the value of `level * size_of_array` is returned. This implementation behaves exactly similar to the current `quantileExact` implementation. diff --git a/docs/ru/sql-reference/aggregate-functions/reference/index.md b/docs/ru/sql-reference/aggregate-functions/reference/index.md index e621e68e8f2..4cbe0a0fba4 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/index.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/index.md @@ -1,68 +1,70 @@ ---- -toc_folder_title: "\u0421\u043f\u0440\u0430\u0432\u043e\u0447\u043d\u0438\u043a" -toc_priority: 36 -toc_hidden: true ---- - -# Перечень агрегатных функций {#aggregate-functions-list} - -Стандартные агрегатные функции: - -- [count](../../../sql-reference/aggregate-functions/reference/count.md) -- [min](../../../sql-reference/aggregate-functions/reference/min.md) -- [max](../../../sql-reference/aggregate-functions/reference/max.md) -- [sum](../../../sql-reference/aggregate-functions/reference/sum.md) -- [avg](../../../sql-reference/aggregate-functions/reference/avg.md) -- [any](../../../sql-reference/aggregate-functions/reference/any.md) -- [stddevPop](../../../sql-reference/aggregate-functions/reference/stddevpop.md) -- [stddevSamp](../../../sql-reference/aggregate-functions/reference/stddevsamp.md) -- [varPop](../../../sql-reference/aggregate-functions/reference/varpop.md) -- [varSamp](../../../sql-reference/aggregate-functions/reference/varsamp.md) -- [covarPop](../../../sql-reference/aggregate-functions/reference/covarpop.md) -- [covarSamp](../../../sql-reference/aggregate-functions/reference/covarsamp.md) - -Агрегатные функции, специфичные для ClickHouse: - -- [anyHeavy](../../../sql-reference/aggregate-functions/reference/anyheavy.md) -- [anyLast](../../../sql-reference/aggregate-functions/reference/anylast.md) -- [argMin](../../../sql-reference/aggregate-functions/reference/argmin.md) -- [argMax](../../../sql-reference/aggregate-functions/reference/argmax.md) -- [avgWeighted](../../../sql-reference/aggregate-functions/reference/avgweighted.md) -- [topK](../../../sql-reference/aggregate-functions/reference/topk.md) -- [topKWeighted](../../../sql-reference/aggregate-functions/reference/topkweighted.md) -- [groupArray](../../../sql-reference/aggregate-functions/reference/grouparray.md) -- [groupUniqArray](../../../sql-reference/aggregate-functions/reference/groupuniqarray.md) -- [groupArrayInsertAt](../../../sql-reference/aggregate-functions/reference/grouparrayinsertat.md) -- [groupArrayMovingAvg](../../../sql-reference/aggregate-functions/reference/grouparraymovingavg.md) -- [groupArrayMovingSum](../../../sql-reference/aggregate-functions/reference/grouparraymovingsum.md) -- [groupBitAnd](../../../sql-reference/aggregate-functions/reference/groupbitand.md) -- [groupBitOr](../../../sql-reference/aggregate-functions/reference/groupbitor.md) -- [groupBitXor](../../../sql-reference/aggregate-functions/reference/groupbitxor.md) -- [groupBitmap](../../../sql-reference/aggregate-functions/reference/groupbitmap.md) -- [sumWithOverflow](../../../sql-reference/aggregate-functions/reference/sumwithoverflow.md) -- [sumMap](../../../sql-reference/aggregate-functions/reference/summap.md) -- [skewSamp](../../../sql-reference/aggregate-functions/reference/skewsamp.md) -- [skewPop](../../../sql-reference/aggregate-functions/reference/skewpop.md) -- [kurtSamp](../../../sql-reference/aggregate-functions/reference/kurtsamp.md) -- [kurtPop](../../../sql-reference/aggregate-functions/reference/kurtpop.md) -- [timeSeriesGroupSum](../../../sql-reference/aggregate-functions/reference/timeseriesgroupsum.md) -- [timeSeriesGroupRateSum](../../../sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md) -- [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md) -- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md) -- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md) -- [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md) -- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md) -- [quantile](../../../sql-reference/aggregate-functions/reference/quantile.md) -- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md) -- [quantileExact](../../../sql-reference/aggregate-functions/reference/quantileexact.md) -- [quantileExactWeighted](../../../sql-reference/aggregate-functions/reference/quantileexactweighted.md) -- [quantileTiming](../../../sql-reference/aggregate-functions/reference/quantiletiming.md) -- [quantileTimingWeighted](../../../sql-reference/aggregate-functions/reference/quantiletimingweighted.md) -- [quantileDeterministic](../../../sql-reference/aggregate-functions/reference/quantiledeterministic.md) -- [quantileTDigest](../../../sql-reference/aggregate-functions/reference/quantiletdigest.md) -- [quantileTDigestWeighted](../../../sql-reference/aggregate-functions/reference/quantiletdigestweighted.md) -- [simpleLinearRegression](../../../sql-reference/aggregate-functions/reference/simplelinearregression.md) -- [stochasticLinearRegression](../../../sql-reference/aggregate-functions/reference/stochasticlinearregression.md) -- [stochasticLogisticRegression](../../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md) - -[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/aggregate-functions/reference) +--- +toc_folder_title: "\u0421\u043f\u0440\u0430\u0432\u043e\u0447\u043d\u0438\u043a" +toc_priority: 36 +toc_hidden: true +--- + +# Перечень агрегатных функций {#aggregate-functions-list} + +Стандартные агрегатные функции: + +- [count](../../../sql-reference/aggregate-functions/reference/count.md) +- [min](../../../sql-reference/aggregate-functions/reference/min.md) +- [max](../../../sql-reference/aggregate-functions/reference/max.md) +- [sum](../../../sql-reference/aggregate-functions/reference/sum.md) +- [avg](../../../sql-reference/aggregate-functions/reference/avg.md) +- [any](../../../sql-reference/aggregate-functions/reference/any.md) +- [stddevPop](../../../sql-reference/aggregate-functions/reference/stddevpop.md) +- [stddevSamp](../../../sql-reference/aggregate-functions/reference/stddevsamp.md) +- [varPop](../../../sql-reference/aggregate-functions/reference/varpop.md) +- [varSamp](../../../sql-reference/aggregate-functions/reference/varsamp.md) +- [covarPop](../../../sql-reference/aggregate-functions/reference/covarpop.md) +- [covarSamp](../../../sql-reference/aggregate-functions/reference/covarsamp.md) + +Агрегатные функции, специфичные для ClickHouse: + +- [anyHeavy](../../../sql-reference/aggregate-functions/reference/anyheavy.md) +- [anyLast](../../../sql-reference/aggregate-functions/reference/anylast.md) +- [argMin](../../../sql-reference/aggregate-functions/reference/argmin.md) +- [argMax](../../../sql-reference/aggregate-functions/reference/argmax.md) +- [avgWeighted](../../../sql-reference/aggregate-functions/reference/avgweighted.md) +- [topK](../../../sql-reference/aggregate-functions/reference/topk.md) +- [topKWeighted](../../../sql-reference/aggregate-functions/reference/topkweighted.md) +- [groupArray](../../../sql-reference/aggregate-functions/reference/grouparray.md) +- [groupUniqArray](../../../sql-reference/aggregate-functions/reference/groupuniqarray.md) +- [groupArrayInsertAt](../../../sql-reference/aggregate-functions/reference/grouparrayinsertat.md) +- [groupArrayMovingAvg](../../../sql-reference/aggregate-functions/reference/grouparraymovingavg.md) +- [groupArrayMovingSum](../../../sql-reference/aggregate-functions/reference/grouparraymovingsum.md) +- [groupBitAnd](../../../sql-reference/aggregate-functions/reference/groupbitand.md) +- [groupBitOr](../../../sql-reference/aggregate-functions/reference/groupbitor.md) +- [groupBitXor](../../../sql-reference/aggregate-functions/reference/groupbitxor.md) +- [groupBitmap](../../../sql-reference/aggregate-functions/reference/groupbitmap.md) +- [sumWithOverflow](../../../sql-reference/aggregate-functions/reference/sumwithoverflow.md) +- [sumMap](../../../sql-reference/aggregate-functions/reference/summap.md) +- [skewSamp](../../../sql-reference/aggregate-functions/reference/skewsamp.md) +- [skewPop](../../../sql-reference/aggregate-functions/reference/skewpop.md) +- [kurtSamp](../../../sql-reference/aggregate-functions/reference/kurtsamp.md) +- [kurtPop](../../../sql-reference/aggregate-functions/reference/kurtpop.md) +- [timeSeriesGroupSum](../../../sql-reference/aggregate-functions/reference/timeseriesgroupsum.md) +- [timeSeriesGroupRateSum](../../../sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md) +- [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md) +- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md) +- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md) +- [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md) +- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md) +- [quantile](../../../sql-reference/aggregate-functions/reference/quantile.md) +- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md) +- [quantileExact](../../../sql-reference/aggregate-functions/reference/quantileexact.md) +- [quantileExactLow](../../../sql-reference/aggregate-functions/reference/quantileexact.md#quantileexactlow) +- [quantileExactHigh](../../../sql-reference/aggregate-functions/reference/quantileexact.md#quantileexacthigh) +- [quantileExactWeighted](../../../sql-reference/aggregate-functions/reference/quantileexactweighted.md) +- [quantileTiming](../../../sql-reference/aggregate-functions/reference/quantiletiming.md) +- [quantileTimingWeighted](../../../sql-reference/aggregate-functions/reference/quantiletimingweighted.md) +- [quantileDeterministic](../../../sql-reference/aggregate-functions/reference/quantiledeterministic.md) +- [quantileTDigest](../../../sql-reference/aggregate-functions/reference/quantiletdigest.md) +- [quantileTDigestWeighted](../../../sql-reference/aggregate-functions/reference/quantiletdigestweighted.md) +- [simpleLinearRegression](../../../sql-reference/aggregate-functions/reference/simplelinearregression.md) +- [stochasticLinearRegression](../../../sql-reference/aggregate-functions/reference/stochasticlinearregression.md) +- [stochasticLogisticRegression](../../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md) + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/aggregate-functions/reference) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/quantileexact.md b/docs/ru/sql-reference/aggregate-functions/reference/quantileexact.md index 49415d96af9..4ee815a94fb 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/quantileexact.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/quantileexact.md @@ -1,56 +1,166 @@ ---- -toc_priority: 202 ---- - -# quantileExact {#quantileexact} - -Точно вычисляет [квантиль](https://ru.wikipedia.org/wiki/Квантиль) числовой последовательности. - -Чтобы получить точный результат, все переданные значения собираются в массив, который затем частично сортируется. Таким образом, функция потребляет объем памяти `O(n)`, где `n` — количество переданных значений. Для небольшого числа значений эта функция эффективна. - -Внутренние состояния функций `quantile*` не объединяются, если они используются в одном запросе. Если вам необходимо вычислить квантили нескольких уровней, используйте функцию [quantiles](#quantiles), это повысит эффективность запроса. - -**Синтаксис** - -``` sql -quantileExact(level)(expr) -``` - -Алиас: `medianExact`. - -**Параметры** - -- `level` — Уровень квантили. Опционально. Константное значение с плавающей запятой от 0 до 1. Мы рекомендуем использовать значение `level` из диапазона `[0.01, 0.99]`. Значение по умолчанию: 0.5. При `level=0.5` функция вычисляет [медиану](https://ru.wikipedia.org/wiki/Медиана_(статистика)). -- `expr` — Выражение над значениями столбца, которое возвращает данные [числовых типов](../../../sql-reference/data-types/index.md#data_types) или типов [Date](../../../sql-reference/data-types/date.md), [DateTime](../../../sql-reference/data-types/datetime.md). - -**Возвращаемое значение** - -- Квантиль заданного уровня. - -Тип: - -- [Float64](../../../sql-reference/data-types/float.md) для входных данных числового типа. -- [Date](../../../sql-reference/data-types/date.md), если входные значения имеют тип `Date`. -- [DateTime](../../../sql-reference/data-types/datetime.md), если входные значения имеют тип `DateTime`. -**Пример** - -Запрос: - -``` sql -SELECT quantileExact(number) FROM numbers(10) -``` - -Результат: - -``` text -┌─quantileExact(number)─┐ -│ 5 │ -└───────────────────────┘ -``` - -**Смотрите также** - -- [median](../../../sql-reference/aggregate-functions/reference/median.md#median) -- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) - -[Оригинальная статья](https://clickhouse.tech/docs/en/sql-reference/aggregate-functions/reference/quantileexact/) +--- +toc_priority: 202 +--- + +# quantileExact {#quantileexact} + +Точно вычисляет [квантиль](https://ru.wikipedia.org/wiki/Квантиль) числовой последовательности. + +Чтобы получить точный результат, все переданные значения собираются в массив, который затем частично сортируется. Таким образом, функция потребляет объем памяти `O(n)`, где `n` — количество переданных значений. Для небольшого числа значений эта функция эффективна. + +Внутренние состояния функций `quantile*` не объединяются, если они используются в одном запросе. Если вам необходимо вычислить квантили нескольких уровней, используйте функцию [quantiles](#quantiles), это повысит эффективность запроса. + +**Синтаксис** + +``` sql +quantileExact(level)(expr) +``` + +Алиас: `medianExact`. + +**Параметры** + +- `level` — Уровень квантили. Опционально. Константное значение с плавающей запятой от 0 до 1. Мы рекомендуем использовать значение `level` из диапазона `[0.01, 0.99]`. Значение по умолчанию: 0.5. При `level=0.5` функция вычисляет [медиану](https://ru.wikipedia.org/wiki/Медиана_(статистика)). +- `expr` — Выражение над значениями столбца, которое возвращает данные [числовых типов](../../../sql-reference/data-types/index.md#data_types) или типов [Date](../../../sql-reference/data-types/date.md), [DateTime](../../../sql-reference/data-types/datetime.md). + +**Возвращаемое значение** + +- Квантиль заданного уровня. + +Тип: + +- [Float64](../../../sql-reference/data-types/float.md) для входных данных числового типа. +- [Date](../../../sql-reference/data-types/date.md), если входные значения имеют тип `Date`. +- [DateTime](../../../sql-reference/data-types/datetime.md), если входные значения имеют тип `DateTime`. + +**Пример** + +Запрос: + +``` sql +SELECT quantileExact(number) FROM numbers(10) +``` + +Результат: + +``` text +┌─quantileExact(number)─┐ +│ 5 │ +└───────────────────────┘ +``` + +# quantileExactLow {#quantileexactlow} + +Как и `quantileExact`, эта функция вычисляет точный [квантиль](https://en.wikipedia.org/wiki/Quantile) числовой последовательности данных. + +Чтобы получить точное значение, все переданные значения объединяются в массив, который затем полностью сортируется. Сложность [алгоритма сортировки](https://en.cppreference.com/w/cpp/algorithm/sort) равна `O(N·log(N))`, где `N = std::distance(first, last)`. + +Возвращаемое значение зависит от уровня квантили и количества элементов в выборке, то есть если уровень 0,5, то функция возвращает нижнюю медиану при чётном количестве элементов и медиану при нечётном. Медиана вычисляется аналогично реализации [median_low](https://docs.python.org/3/library/statistics.html#statistics.median_low), которая используется в python. + +Для всех остальных уровней возвращается элемент с индексом, соответствующим значению `level * size_of_array`. Например: + +``` sql +SELECT quantileExactLow(0.1)(number) FROM numbers(10) + +┌─quantileExactLow(0.1)(number)─┐ +│ 1 │ +└───────────────────────────────┘ +``` + +При использовании в запросе нескольких функций `quantile*` с разными уровнями, внутренние состояния не объединяются (то есть запрос работает менее эффективно). В этом случае используйте функцию [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles). + +**Синтаксис** + +``` sql +quantileExact(level)(expr) +``` + +Алиас: `medianExactLow`. + +**Параметры** + +- `level` — Уровень квантили. Опциональный параметр. Константное занчение с плавающей запятой от 0 до 1. Мы рекомендуем использовать значение `level` из диапазона `[0.01, 0.99]`. Значение по умолчанию: 0.5. При `level=0.5` функция вычисляет [медиану](https://en.wikipedia.org/wiki/Median). +- `expr` — Выражение над значениями столбца, которое возвращает данные [числовых типов](../../../sql-reference/data-types/index.md#data_types), [Date](../../../sql-reference/data-types/date.md) или [DateTime](../../../sql-reference/data-types/datetime.md). + +**Возвращаемое значение** + +- Квантиль заданного уровня. + +Тип: + +- [Float64](../../../sql-reference/data-types/float.md) для входных данных числового типа. +- [Date](../../../sql-reference/data-types/date.md) если входные значения имеют тип `Date`. +- [DateTime](../../../sql-reference/data-types/datetime.md) если входные значения имеют тип `DateTime`. + +**Пример** + +Запрос: + +``` sql +SELECT quantileExactLow(number) FROM numbers(10) +``` + +Результат: + +``` text +┌─quantileExactLow(number)─┐ +│ 4 │ +└──────────────────────────┘ +``` +# quantileExactHigh {#quantileexacthigh} + +Как и `quantileExact`, эта функция вычисляет точный [квантиль](https://en.wikipedia.org/wiki/Quantile) числовой последовательности данных. + +Все переданные значения объединяются в массив, который затем сортируется, чтобы получить точное значение. Сложность [алгоритма сортировки](https://en.cppreference.com/w/cpp/algorithm/sort) равна `O(N·log(N))`, где `N = std::distance(first, last)`. + +Возвращаемое значение зависит от уровня квантили и количества элементов в выборке, то есть если уровень 0,5, то функция возвращает верхнюю медиану при чётном количестве элементов и медиану при нечётном. Медиана вычисляется аналогично реализации [median_high](https://docs.python.org/3/library/statistics.html#statistics.median_high), которая используется в python. Для всех остальных уровней возвращается элемент с индексом, соответствующим значению `level * size_of_array`. + +Эта реализация ведет себя точно так же, как `quantileExact`. + +При использовании в запросе нескольких функций `quantile*` с разными уровнями, внутренние состояния не объединяются (то есть запрос работает менее эффективно). В этом случае используйте функцию [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles). + +**Синтаксис** + +``` sql +quantileExactHigh(level)(expr) +``` + +Алиас: `medianExactHigh`. + +**Параметры** + +- `level` — Уровень квантили. Опциональный параметр. Константное занчение с плавающей запятой от 0 до 1. Мы рекомендуем использовать значение `level` из диапазона `[0.01, 0.99]`. Значение по умолчанию: 0.5. При `level=0.5` функция вычисляет [медиану](https://en.wikipedia.org/wiki/Median). +- `expr` — Выражение над значениями столбца, которое возвращает данные [числовых типов](../../../sql-reference/data-types/index.md#data_types), [Date](../../../sql-reference/data-types/date.md) или [DateTime](../../../sql-reference/data-types/datetime.md). + +**Возвращаемое значение** + +- Квантиль заданного уровня. + +Тип: + +- [Float64](../../../sql-reference/data-types/float.md) для входных данных числового типа. +- [Date](../../../sql-reference/data-types/date.md) если входные значения имеют тип `Date`. +- [DateTime](../../../sql-reference/data-types/datetime.md) если входные значения имеют тип `DateTime`. + +**Пример** + +Запрос: + +``` sql +SELECT quantileExactHigh(number) FROM numbers(10) +``` + +Результат: + +``` text +┌─quantileExactHigh(number)─┐ +│ 5 │ +└───────────────────────────┘ +``` + +**Смотрите также** + +- [median](../../../sql-reference/aggregate-functions/reference/median.md#median) +- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) + +[Оригинальная статья](https://clickhouse.tech/docs/en/sql-reference/aggregate-functions/reference/quantileexact/) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 9367f3be00c..91bf0f5b3a0 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -927,6 +927,48 @@ SELECT defaultValueOfArgumentType( CAST(1 AS Nullable(Int8) ) ) └───────────────────────────────────────────────────────┘ ``` +## defaultValueOfTypeName {#defaultvalueoftypename} + +Выводит значение по умолчанию для указанного типа данных. + +Не включает значения по умолчанию для настраиваемых столбцов, установленных пользователем. + +``` sql +defaultValueOfTypeName(type) +``` + +**Параметры:** + +- `type` — тип данных. + +**Возвращаемое значение** + +- `0` для чисел; +- Пустая строка для строк; +- `ᴺᵁᴸᴸ` для [Nullable](../../sql-reference/data-types/nullable.md). + +**Пример** + +``` sql +SELECT defaultValueOfTypeName('Int8') +``` + +``` text +┌─defaultValueOfTypeName('Int8')─┐ +│ 0 │ +└────────────────────────────────┘ +``` + +``` sql +SELECT defaultValueOfTypeName('Nullable(Int8)') +``` + +``` text +┌─defaultValueOfTypeName('Nullable(Int8)')─┐ +│ ᴺᵁᴸᴸ │ +└──────────────────────────────────────────┘ +``` + ## replicate {#other-functions-replicate} Создает массив, заполненный одним значением. From 0e0b154d244c204d0abf071d4ce9804051de6532 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 4 Nov 2020 23:07:56 +0300 Subject: [PATCH 211/314] Update settings.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Внес поправки согласно комментариям ревьюера. --- .../server-configuration-parameters/settings.md | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 31a8e896438..ad7f92a76f4 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -471,7 +471,7 @@ On hosts with low RAM and swap, you possibly need setting `max_server_memory_usa ## max_concurrent_queries {#max-concurrent-queries} -The maximum number of simultaneously processed requests. +The maximum number of simultaneously processed queries. **Example** @@ -483,11 +483,18 @@ The maximum number of simultaneously processed requests. Throw exception if the value of this setting is less or equal than the current number of simultaneously processed queries. -Example: `max_concurrent_queries_for_all_users` can be set to 99 for all users and database administrator can set it to 100 for itself to run queries for investigation even when the server is overloaded. - Modifying the setting for one query or user does not affect other queries. -Default value: `0` that means no limit. +Possible values: + +- Positive integer. +- 0 — Unlimited. + +Default value: `0`. + +**Usage** + +`max_concurrent_queries_for_all_users` can be set to 99 for all users, and database administrator can set it to 100 or even 0 for himself to run queries for investigation even when the ClickHouse server is overloaded. **Example** From 7dd749fc4c6ae8533b2e4f475899de46ec37f4b2 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 4 Nov 2020 23:41:10 +0300 Subject: [PATCH 212/314] BloomFilter index added big integers and UUID support --- src/Interpreters/BloomFilterHash.h | 16 +++++++++--- src/Interpreters/convertFieldToType.cpp | 4 +++ .../MergeTree/MergeTreeIndexBloomFilter.cpp | 2 +- ...om_filter_index_big_integer_uuid.reference | 12 +++++++++ ...54_bloom_filter_index_big_integer_uuid.sql | 25 +++++++++++++++++++ 5 files changed, 55 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.reference create mode 100644 tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.sql diff --git a/src/Interpreters/BloomFilterHash.h b/src/Interpreters/BloomFilterHash.h index 43f5d7b5e87..b64f7bae79d 100644 --- a/src/Interpreters/BloomFilterHash.h +++ b/src/Interpreters/BloomFilterHash.h @@ -38,7 +38,7 @@ struct BloomFilterHash static UInt64 getNumberTypeHash(const Field & field) { /// For negative, we should convert the type to make sure the symbol is in right place - return field.isNull() ? intHash64(0) : intHash64(ext::bit_cast(FieldType(field.safeGet()))); + return field.isNull() ? intHash64(0) : DefaultHash64(FieldType(field.safeGet())); } static UInt64 getStringTypeHash(const Field & field) @@ -79,16 +79,21 @@ struct BloomFilterHash else if (which.isUInt16()) return build_hash_column(getNumberTypeHash(field)); else if (which.isUInt32()) return build_hash_column(getNumberTypeHash(field)); else if (which.isUInt64()) return build_hash_column(getNumberTypeHash(field)); + else if (which.isUInt128()) return build_hash_column(getNumberTypeHash(field)); + else if (which.isUInt256()) return build_hash_column(getNumberTypeHash(field)); else if (which.isInt8()) return build_hash_column(getNumberTypeHash(field)); else if (which.isInt16()) return build_hash_column(getNumberTypeHash(field)); else if (which.isInt32()) return build_hash_column(getNumberTypeHash(field)); else if (which.isInt64()) return build_hash_column(getNumberTypeHash(field)); + else if (which.isInt128()) return build_hash_column(getNumberTypeHash(field)); + else if (which.isInt256()) return build_hash_column(getNumberTypeHash(field)); else if (which.isEnum8()) return build_hash_column(getNumberTypeHash(field)); else if (which.isEnum16()) return build_hash_column(getNumberTypeHash(field)); else if (which.isDate()) return build_hash_column(getNumberTypeHash(field)); else if (which.isDateTime()) return build_hash_column(getNumberTypeHash(field)); else if (which.isFloat32()) return build_hash_column(getNumberTypeHash(field)); else if (which.isFloat64()) return build_hash_column(getNumberTypeHash(field)); + else if (which.isUUID()) return build_hash_column(getNumberTypeHash(field)); else if (which.isString()) return build_hash_column(getStringTypeHash(field)); else if (which.isFixedString()) return build_hash_column(getFixedStringTypeHash(field, data_type)); else throw Exception("Unexpected type " + data_type->getName() + " of bloom filter index.", ErrorCodes::BAD_ARGUMENTS); @@ -135,16 +140,21 @@ struct BloomFilterHash else if (which.isUInt16()) getNumberTypeHash(column, vec, pos); else if (which.isUInt32()) getNumberTypeHash(column, vec, pos); else if (which.isUInt64()) getNumberTypeHash(column, vec, pos); + else if (which.isUInt128()) getNumberTypeHash(column, vec, pos); + else if (which.isUInt256()) getNumberTypeHash(column, vec, pos); else if (which.isInt8()) getNumberTypeHash(column, vec, pos); else if (which.isInt16()) getNumberTypeHash(column, vec, pos); else if (which.isInt32()) getNumberTypeHash(column, vec, pos); else if (which.isInt64()) getNumberTypeHash(column, vec, pos); + else if (which.isInt128()) getNumberTypeHash(column, vec, pos); + else if (which.isInt256()) getNumberTypeHash(column, vec, pos); else if (which.isEnum8()) getNumberTypeHash(column, vec, pos); else if (which.isEnum16()) getNumberTypeHash(column, vec, pos); else if (which.isDate()) getNumberTypeHash(column, vec, pos); else if (which.isDateTime()) getNumberTypeHash(column, vec, pos); else if (which.isFloat32()) getNumberTypeHash(column, vec, pos); else if (which.isFloat64()) getNumberTypeHash(column, vec, pos); + else if (which.isUUID()) getNumberTypeHash(column, vec, pos); else if (which.isString()) getStringTypeHash(column, vec, pos); else if (which.isFixedString()) getStringTypeHash(column, vec, pos); else throw Exception("Unexpected type " + data_type->getName() + " of bloom filter index.", ErrorCodes::BAD_ARGUMENTS); @@ -166,7 +176,7 @@ struct BloomFilterHash { for (size_t index = 0, size = vec.size(); index < size; ++index) { - UInt64 hash = intHash64(ext::bit_cast(Float64(vec_from[index + pos]))); + UInt64 hash = DefaultHash64(Float64(vec_from[index + pos])); if constexpr (is_first) vec[index] = hash; @@ -178,7 +188,7 @@ struct BloomFilterHash { for (size_t index = 0, size = vec.size(); index < size; ++index) { - UInt64 hash = intHash64(ext::bit_cast(vec_from[index + pos])); + UInt64 hash = DefaultHash64(vec_from[index + pos]); if constexpr (is_first) vec[index] = hash; diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index c0e9d7edc13..18a3eeecb46 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -152,10 +152,14 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID if (which_type.isUInt16()) return convertNumericType(src, type); if (which_type.isUInt32()) return convertNumericType(src, type); if (which_type.isUInt64()) return convertNumericType(src, type); + if (which_type.isUInt128()) return convertNumericType(src, type); + if (which_type.isUInt256()) return convertNumericType(src, type); if (which_type.isInt8()) return convertNumericType(src, type); if (which_type.isInt16()) return convertNumericType(src, type); if (which_type.isInt32()) return convertNumericType(src, type); if (which_type.isInt64()) return convertNumericType(src, type); + if (which_type.isInt128()) return convertNumericType(src, type); + if (which_type.isInt256()) return convertNumericType(src, type); if (which_type.isFloat32()) return convertNumericType(src, type); if (which_type.isFloat64()) return convertNumericType(src, type); if (const auto * ptype = typeid_cast *>(&type)) return convertDecimalType(src, *ptype); diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index 5a5dfffe287..a98ba16978d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -85,7 +85,7 @@ static void assertIndexColumnsType(const Block & header) WhichDataType which(actual_type); if (!which.isUInt() && !which.isInt() && !which.isString() && !which.isFixedString() && !which.isFloat() && - !which.isDateOrDateTime() && !which.isEnum()) + !which.isDateOrDateTime() && !which.isEnum() && !which.isUUID()) throw Exception("Unexpected type " + type->getName() + " of bloom filter index.", ErrorCodes::ILLEGAL_COLUMN); } diff --git a/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.reference b/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.reference new file mode 100644 index 00000000000..39beae69d65 --- /dev/null +++ b/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.reference @@ -0,0 +1,12 @@ +1 +1 +2 +1 +1 +2 +1 +1 +2 +00000000-0000-0001-0000-000000000000 +00000000-0000-0001-0000-000000000000 +00000000-0000-0002-0000-000000000000 diff --git a/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.sql b/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.sql new file mode 100644 index 00000000000..265f931c556 --- /dev/null +++ b/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.sql @@ -0,0 +1,25 @@ +SET allow_experimental_bigint_types = 1; + +CREATE TABLE 01154_test (x Int128, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192; +INSERT INTO 01154_test VALUES (1), (2), (3); +SELECT x FROM 01154_test WHERE x = 1; +SELECT x FROM 01154_test WHERE x IN (1, 2); +DROP TABLE 01154_test; + +CREATE TABLE 01154_test (x Int256, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192; +INSERT INTO 01154_test VALUES (1), (2), (3); +SELECT x FROM 01154_test WHERE x = 1; +SELECT x FROM 01154_test WHERE x IN (1, 2); +DROP TABLE 01154_test; + +CREATE TABLE 01154_test (x UInt256, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192; +INSERT INTO 01154_test VALUES (1), (2), (3); +SELECT x FROM 01154_test WHERE x = 1; +SELECT x FROM 01154_test WHERE x IN (1, 2); +DROP TABLE 01154_test; + +CREATE TABLE 01154_test (x UUID, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192; +INSERT INTO 01154_test VALUES (toUUID(1)), (toUUID(2)), (toUUID(3)); +SELECT x FROM 01154_test WHERE x = toUUID(1); +SELECT x FROM 01154_test WHERE x IN (toUUID(1), toUUID(2)); +DROP TABLE 01154_test; From cd19509b5c2f1d2e538934b03ae5aa2f23eff12f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 4 Nov 2020 23:42:17 +0300 Subject: [PATCH 213/314] Update 01541_max_memory_usage_for_user.sh --- tests/queries/0_stateless/01541_max_memory_usage_for_user.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01541_max_memory_usage_for_user.sh b/tests/queries/0_stateless/01541_max_memory_usage_for_user.sh index 3b1ced03473..64a90d871e7 100755 --- a/tests/queries/0_stateless/01541_max_memory_usage_for_user.sh +++ b/tests/queries/0_stateless/01541_max_memory_usage_for_user.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # - one users' query in background (to avoid reseting max_memory_usage_for_user) # We have to create a separate user to run this tests isolated from other tests. -${CLICKHOUSE_CLIENT} -n --query "DROP USER IF EXISTS test_01541; CREATE USER test_01541; GRANT ALL ON *.* TO test_01541;"; +${CLICKHOUSE_CLIENT} -n --allow_introspection_functions 1 --query "DROP USER IF EXISTS test_01541; CREATE USER test_01541; GRANT ALL ON *.* TO test_01541;"; query="SELECT groupArray(repeat('a', 1000)) FROM numbers(10000) GROUP BY number % 10 FORMAT JSON" From b1e75ec6f5b34de8ac8d53f8f7a4a1110edd2f84 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 4 Nov 2020 23:43:11 +0300 Subject: [PATCH 214/314] CreateQuery compound column type indendation fix --- src/Parsers/ASTColumnDeclaration.cpp | 6 +++++- .../0_stateless/01458_named_tuple_millin.reference | 4 ++-- .../queries/0_stateless/01458_named_tuple_millin.sql | 4 +--- ...548_create_table_compound_column_format.reference | 12 ++++++++++++ .../01548_create_table_compound_column_format.sh | 8 ++++++++ 5 files changed, 28 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/01548_create_table_compound_column_format.reference create mode 100755 tests/queries/0_stateless/01548_create_table_compound_column_format.sh diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 730e892f8f7..0e0847713c2 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -55,7 +55,11 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & settings, FormatSta if (type) { settings.ostr << ' '; - type->formatImpl(settings, state, frame); + + FormatStateStacked typeFrame = frame; + typeFrame.indent = 0; + + type->formatImpl(settings, state, typeFrame); } if (null_modifier) diff --git a/tests/queries/0_stateless/01458_named_tuple_millin.reference b/tests/queries/0_stateless/01458_named_tuple_millin.reference index b826566c74b..d6d6d7ae8d4 100644 --- a/tests/queries/0_stateless/01458_named_tuple_millin.reference +++ b/tests/queries/0_stateless/01458_named_tuple_millin.reference @@ -1,12 +1,12 @@ CREATE TABLE default.tuple ( - `j` Tuple( a Int8, b String) + `j` Tuple(a Int8, b String) ) ENGINE = Memory j Tuple(a Int8, b String) CREATE TABLE default.tuple ( - `j` Tuple( a Int8, b String) + `j` Tuple(a Int8, b String) ) ENGINE = Memory j Tuple(a Int8, b String) diff --git a/tests/queries/0_stateless/01458_named_tuple_millin.sql b/tests/queries/0_stateless/01458_named_tuple_millin.sql index 7687dd4c158..ea730e65bb7 100644 --- a/tests/queries/0_stateless/01458_named_tuple_millin.sql +++ b/tests/queries/0_stateless/01458_named_tuple_millin.sql @@ -10,9 +10,7 @@ SHOW CREATE TABLE tuple FORMAT TSVRaw; DESC tuple; DROP TABLE tuple; -CREATE TABLE tuple -ENGINE = Memory AS -SELECT CAST((1, 'Test'), 'Tuple(a Int8, b String)') AS j; +CREATE TABLE tuple ENGINE = Memory AS SELECT CAST((1, 'Test'), 'Tuple(a Int8, b String)') AS j; SHOW CREATE TABLE tuple FORMAT TSVRaw; DESC tuple; diff --git a/tests/queries/0_stateless/01548_create_table_compound_column_format.reference b/tests/queries/0_stateless/01548_create_table_compound_column_format.reference new file mode 100644 index 00000000000..c6c4dcdfa4a --- /dev/null +++ b/tests/queries/0_stateless/01548_create_table_compound_column_format.reference @@ -0,0 +1,12 @@ +CREATE TABLE test +( + `a` Int64, + `b` NESTED(a Int64) +) +ENGINE = TinyLog +CREATE TABLE test +( + `a` Int64, + `b` TUPLE(a Int64) +) +ENGINE = TinyLog diff --git a/tests/queries/0_stateless/01548_create_table_compound_column_format.sh b/tests/queries/0_stateless/01548_create_table_compound_column_format.sh new file mode 100755 index 00000000000..6c9384e01c1 --- /dev/null +++ b/tests/queries/0_stateless/01548_create_table_compound_column_format.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +echo "CREATE TABLE test(a Int64, b NESTED(a Int64)) ENGINE=TinyLog" | $CLICKHOUSE_FORMAT + +echo "CREATE TABLE test(a Int64, b TUPLE(a Int64)) ENGINE=TinyLog" | $CLICKHOUSE_FORMAT \ No newline at end of file From d5a51ddbbfef08b305f592e90af1b116c1eeac94 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 4 Nov 2020 23:44:43 +0300 Subject: [PATCH 215/314] Update CMakeLists.txt --- programs/odbc-bridge/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/odbc-bridge/CMakeLists.txt b/programs/odbc-bridge/CMakeLists.txt index 043c0d5dd7d..8e7ccfedc70 100644 --- a/programs/odbc-bridge/CMakeLists.txt +++ b/programs/odbc-bridge/CMakeLists.txt @@ -23,7 +23,7 @@ add_executable(clickhouse-odbc-bridge ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}) target_link_libraries(clickhouse-odbc-bridge PRIVATE daemon - clickhouse_common_io + dbms Poco::Data Poco::Data::ODBC ) From fa929b03122c8724f7cd28a78d8d338f14b6ab90 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 4 Nov 2020 23:45:06 +0300 Subject: [PATCH 216/314] Replace duplicate development config files with symlinks. (#16486) * Fix test "max_memory_usage_for_user" * Update test * Update annoying boilerplate * Support symlinks in tarballs * Fix Fuzzer * Remove "secondary" user * Remove "secondary" user --- docker/packager/binary/build.sh | 2 +- docker/test/fuzzer/run-fuzzer.sh | 10 ++++----- programs/server/config.d/macros.xml | 6 +---- programs/server/config.d/metric_log.xml | 9 +------- programs/server/config.d/part_log.xml | 8 +------ .../server/config.d/query_masking_rules.xml | 20 +---------------- programs/server/config.d/text_log.xml | 8 +------ programs/server/config.d/zookeeper.xml | 6 +---- programs/server/users.d/access_management.xml | 8 +------ programs/server/users.d/log_queries.xml | 9 +------- programs/server/users.d/readonly.xml | 22 +------------------ 11 files changed, 15 insertions(+), 93 deletions(-) mode change 100644 => 120000 programs/server/config.d/macros.xml mode change 100644 => 120000 programs/server/config.d/metric_log.xml mode change 100644 => 120000 programs/server/config.d/part_log.xml mode change 100644 => 120000 programs/server/config.d/query_masking_rules.xml mode change 100644 => 120000 programs/server/config.d/text_log.xml mode change 100644 => 120000 programs/server/config.d/zookeeper.xml mode change 100644 => 120000 programs/server/users.d/access_management.xml mode change 100644 => 120000 programs/server/users.d/log_queries.xml mode change 100644 => 120000 programs/server/users.d/readonly.xml diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 8acbe271b1f..4ba2f5f87c7 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -63,7 +63,7 @@ then mkdir -p /output/config cp ../programs/server/config.xml /output/config cp ../programs/server/users.xml /output/config - cp -r ../programs/server/config.d /output/config + cp -r --dereference ../programs/server/config.d /output/config tar -czvf "$COMBINED_OUTPUT.tgz" /output rm -r /output/* mv "$COMBINED_OUTPUT.tgz" /output diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 41559a9b1f5..26db6455fd5 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -45,11 +45,11 @@ function configure { rm -rf db ||: mkdir db ||: - cp -av "$repo_dir"/programs/server/config* db - cp -av "$repo_dir"/programs/server/user* db + cp -av --dereference "$repo_dir"/programs/server/config* db + cp -av --dereference "$repo_dir"/programs/server/user* db # TODO figure out which ones are needed - cp -av "$repo_dir"/tests/config/config.d/listen.xml db/config.d - cp -av "$script_dir"/query-fuzzer-tweaks-users.xml db/users.d + cp -av --dereference "$repo_dir"/tests/config/config.d/listen.xml db/config.d + cp -av --dereference "$script_dir"/query-fuzzer-tweaks-users.xml db/users.d } function watchdog @@ -89,7 +89,7 @@ function fuzz > >(tail -10000 > fuzzer.log) \ 2>&1 \ || fuzzer_exit_code=$? - + echo "Fuzzer exit code is $fuzzer_exit_code" ./clickhouse-client --query "select elapsed, query from system.processes" ||: diff --git a/programs/server/config.d/macros.xml b/programs/server/config.d/macros.xml deleted file mode 100644 index 1f86f5f9efd..00000000000 --- a/programs/server/config.d/macros.xml +++ /dev/null @@ -1,5 +0,0 @@ - - - Hello, world! - - diff --git a/programs/server/config.d/macros.xml b/programs/server/config.d/macros.xml new file mode 120000 index 00000000000..f28bafb97fa --- /dev/null +++ b/programs/server/config.d/macros.xml @@ -0,0 +1 @@ +../../../tests/config/config.d/macros.xml \ No newline at end of file diff --git a/programs/server/config.d/metric_log.xml b/programs/server/config.d/metric_log.xml deleted file mode 100644 index 0ca9f162416..00000000000 --- a/programs/server/config.d/metric_log.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - system - metric_log
- 7500 - 1000 -
-
diff --git a/programs/server/config.d/metric_log.xml b/programs/server/config.d/metric_log.xml new file mode 120000 index 00000000000..7f033c60a64 --- /dev/null +++ b/programs/server/config.d/metric_log.xml @@ -0,0 +1 @@ +../../../tests/config/config.d/metric_log.xml \ No newline at end of file diff --git a/programs/server/config.d/part_log.xml b/programs/server/config.d/part_log.xml deleted file mode 100644 index 35add3c6cc1..00000000000 --- a/programs/server/config.d/part_log.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - system - part_log
- 7500 -
-
diff --git a/programs/server/config.d/part_log.xml b/programs/server/config.d/part_log.xml new file mode 120000 index 00000000000..d97ea7f226d --- /dev/null +++ b/programs/server/config.d/part_log.xml @@ -0,0 +1 @@ +../../../tests/config/config.d/part_log.xml \ No newline at end of file diff --git a/programs/server/config.d/query_masking_rules.xml b/programs/server/config.d/query_masking_rules.xml deleted file mode 100644 index f919523472c..00000000000 --- a/programs/server/config.d/query_masking_rules.xml +++ /dev/null @@ -1,19 +0,0 @@ - - - - - - - profanity - (?i:shit) - substance - - - - TOPSECRET.TOPSECRET - [hidden] - - - diff --git a/programs/server/config.d/query_masking_rules.xml b/programs/server/config.d/query_masking_rules.xml new file mode 120000 index 00000000000..2e2a234a67b --- /dev/null +++ b/programs/server/config.d/query_masking_rules.xml @@ -0,0 +1 @@ +../../../tests/config/config.d/query_masking_rules.xml \ No newline at end of file diff --git a/programs/server/config.d/text_log.xml b/programs/server/config.d/text_log.xml deleted file mode 100644 index 3699a23578c..00000000000 --- a/programs/server/config.d/text_log.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - system - text_log
- 7500 -
-
diff --git a/programs/server/config.d/text_log.xml b/programs/server/config.d/text_log.xml new file mode 120000 index 00000000000..49203862fa6 --- /dev/null +++ b/programs/server/config.d/text_log.xml @@ -0,0 +1 @@ +../../../tests/config/config.d/text_log.xml \ No newline at end of file diff --git a/programs/server/config.d/zookeeper.xml b/programs/server/config.d/zookeeper.xml deleted file mode 100644 index 68c85788c98..00000000000 --- a/programs/server/config.d/zookeeper.xml +++ /dev/null @@ -1,5 +0,0 @@ - - - testkeeper - - diff --git a/programs/server/config.d/zookeeper.xml b/programs/server/config.d/zookeeper.xml new file mode 120000 index 00000000000..921e0011378 --- /dev/null +++ b/programs/server/config.d/zookeeper.xml @@ -0,0 +1 @@ +../../../tests/config/config.d/zookeeper.xml \ No newline at end of file diff --git a/programs/server/users.d/access_management.xml b/programs/server/users.d/access_management.xml deleted file mode 100644 index 7e799cb7b10..00000000000 --- a/programs/server/users.d/access_management.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - - 1 - - - diff --git a/programs/server/users.d/access_management.xml b/programs/server/users.d/access_management.xml new file mode 120000 index 00000000000..b7331126e09 --- /dev/null +++ b/programs/server/users.d/access_management.xml @@ -0,0 +1 @@ +../../../tests/config/users.d/access_management.xml \ No newline at end of file diff --git a/programs/server/users.d/log_queries.xml b/programs/server/users.d/log_queries.xml deleted file mode 100644 index 796a115d8a6..00000000000 --- a/programs/server/users.d/log_queries.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - - - 1 - - - diff --git a/programs/server/users.d/log_queries.xml b/programs/server/users.d/log_queries.xml new file mode 120000 index 00000000000..c7c0c99e000 --- /dev/null +++ b/programs/server/users.d/log_queries.xml @@ -0,0 +1 @@ +../../../tests/config/users.d/log_queries.xml \ No newline at end of file diff --git a/programs/server/users.d/readonly.xml b/programs/server/users.d/readonly.xml deleted file mode 100644 index 64fbaf77464..00000000000 --- a/programs/server/users.d/readonly.xml +++ /dev/null @@ -1,21 +0,0 @@ - - - - - - 1 - - - - - - - - ::1 - 127.0.0.1 - - readonly - default - - - diff --git a/programs/server/users.d/readonly.xml b/programs/server/users.d/readonly.xml new file mode 120000 index 00000000000..9029a2ce480 --- /dev/null +++ b/programs/server/users.d/readonly.xml @@ -0,0 +1 @@ +../../../tests/config/users.d/readonly.xml \ No newline at end of file From 6757fb69960a5d75513da685c319c87acfdf1126 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Nov 2020 01:17:19 +0300 Subject: [PATCH 217/314] Add development roadmap for web UI --- programs/server/play.html | 26 +++++++++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/programs/server/play.html b/programs/server/play.html index de918191cdf..22eea0002ca 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -3,7 +3,7 @@ ClickHouse Query - + +