From 6426960af99e6c5232b79ea9c2588bc0b27265a0 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Mon, 13 Sep 2021 21:06:36 +0300 Subject: [PATCH 001/173] add new column --- docs/en/operations/system-tables/replicas.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/system-tables/replicas.md b/docs/en/operations/system-tables/replicas.md index e2cc607f6d8..6cdd1ecdce1 100644 --- a/docs/en/operations/system-tables/replicas.md +++ b/docs/en/operations/system-tables/replicas.md @@ -83,6 +83,7 @@ The next 4 columns have a non-zero value only where there is an active session w - `total_replicas` (`UInt8`) - The total number of known replicas of this table. - `active_replicas` (`UInt8`) - The number of replicas of this table that have a session in ZooKeeper (i.e., the number of functioning replicas). - `replica_is_active` ([Map(String, UInt8)](../../sql-reference/data-types/map.md)) — Map between replica name and is replica active. +- `last_queue_update_exception`- When the queue contains broken entries. Especially important when ClickHouse breaks backward compatibility between versions and log entries written by newer versions aren't parseable by old versions. If you request all the columns, the table may work a bit slowly, since several reads from ZooKeeper are made for each row. If you do not request the last 4 columns (log_max_index, log_pointer, total_replicas, active_replicas), the table works quickly. From 0f030f868fd5f1c2bba58f458da0f02b88017143 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Mon, 13 Sep 2021 21:17:31 +0300 Subject: [PATCH 002/173] add type --- docs/en/operations/system-tables/replicas.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/replicas.md b/docs/en/operations/system-tables/replicas.md index 6cdd1ecdce1..251c7e65537 100644 --- a/docs/en/operations/system-tables/replicas.md +++ b/docs/en/operations/system-tables/replicas.md @@ -83,7 +83,7 @@ The next 4 columns have a non-zero value only where there is an active session w - `total_replicas` (`UInt8`) - The total number of known replicas of this table. - `active_replicas` (`UInt8`) - The number of replicas of this table that have a session in ZooKeeper (i.e., the number of functioning replicas). - `replica_is_active` ([Map(String, UInt8)](../../sql-reference/data-types/map.md)) — Map between replica name and is replica active. -- `last_queue_update_exception`- When the queue contains broken entries. Especially important when ClickHouse breaks backward compatibility between versions and log entries written by newer versions aren't parseable by old versions. +- `last_queue_update_exception` (`String`) - When the queue contains broken entries. Especially important when ClickHouse breaks backward compatibility between versions and log entries written by newer versions aren't parseable by old versions. If you request all the columns, the table may work a bit slowly, since several reads from ZooKeeper are made for each row. If you do not request the last 4 columns (log_max_index, log_pointer, total_replicas, active_replicas), the table works quickly. From c428f433c3a2bfab7fc80036c8e36171ce57bec0 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Thu, 16 Sep 2021 16:38:27 +0100 Subject: [PATCH 003/173] Use do-while control loops for waiting on log processing This allows to use the same functions with very short timeouts while ensuring that the actual state is checked at least once instead of timing out before even looking at at ZK at least once. --- src/Common/ZooKeeper/ZooKeeper.cpp | 8 ++++++-- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++----- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 220b52104b5..56b96a3a45c 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -730,7 +730,10 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & } }; - while (!condition || !condition()) + /// do-while control structure to allow using this function in non-blocking + /// fashion with a wait condition which returns false by the time this + /// method is called. + do { /// Use getData insteand of exists to avoid watch leak. impl->get(path, callback, watch); @@ -746,7 +749,8 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & if (state->event_type == Coordination::DELETED) return true; - } + } while (!condition || !condition()); + return false; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e0265ec0b09..47d5a1bc895 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5105,8 +5105,7 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry( /// Don't recheck ZooKeeper too often constexpr auto event_wait_timeout_ms = 3000; - if (!startsWith(entry.znode_name, "log-")) - throw Exception("Logical error: unexpected name of log node: " + entry.znode_name, ErrorCodes::LOGICAL_ERROR); + LOG_DEBUG(log, "Waiting for {} to process log entry", replica); { /// Take the number from the node name `log-xxxxxxxxxx`. @@ -5117,7 +5116,7 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry( /// Let's wait until entry gets into the replica queue. bool pulled_to_queue = false; - while (!stop_waiting()) + do { zkutil::EventPtr event = std::make_shared(); @@ -5132,8 +5131,9 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry( /// So log_pointer node will exist, but we will never update it because all background threads already stopped. /// It can lead to query hung because table drop query can wait for some query (alter, optimize, etc) which called this method, /// but the query will never finish because the drop already shut down the table. - event->tryWait(event_wait_timeout_ms); - } + if (!stop_waiting()) + event->tryWait(event_wait_timeout_ms); + } while (!stop_waiting()); if (!pulled_to_queue) return false; From 0381c634d4fbdbfbb1f88bdefc5a3545eda9ca89 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Thu, 16 Sep 2021 16:47:57 +0100 Subject: [PATCH 004/173] Add support for user defined identifier on log entries Sometimes we want to push a log entry once and only once. Because it is not possible to create a sequential node in ZooKeeper and store its name to a well known location in the same transaction we'll do it in the other order. First somehow generate a unique identifier, then submit a log entry with that identifier. Later, we can search through log entries using the identifier we provided to find the node. Required for part movement between shards. --- .../MergeTree/ReplicatedMergeTreeLogEntry.cpp | 10 +++ .../MergeTree/ReplicatedMergeTreeLogEntry.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 87 ++++++++++++++++++- 3 files changed, 95 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index 18e90952721..89515b863ca 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -26,6 +26,7 @@ enum FormatVersion : UInt8 FORMAT_WITH_DEDUPLICATE = 4, FORMAT_WITH_UUID = 5, FORMAT_WITH_DEDUPLICATE_BY_COLUMNS = 6, + FORMAT_WITH_LOG_ENTRY_ID = 7, FORMAT_LAST }; @@ -43,11 +44,17 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const if (new_part_uuid != UUIDHelpers::Nil) format_version = std::max(format_version, FORMAT_WITH_UUID); + if (!log_entry_id.empty()) + format_version = std::max(format_version, FORMAT_WITH_LOG_ENTRY_ID); + out << "format version: " << format_version << "\n" << "create_time: " << LocalDateTime(create_time ? create_time : time(nullptr)) << "\n" << "source replica: " << source_replica << '\n' << "block_id: " << escape << block_id << '\n'; + if (format_version >= FORMAT_WITH_LOG_ENTRY_ID) + out << "log_entry_id: " << escape << log_entry_id << '\n'; + switch (type) { case GET_PART: @@ -192,6 +199,9 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) in >> "block_id: " >> escape >> block_id >> "\n"; } + if (format_version >= FORMAT_WITH_LOG_ENTRY_ID) + in >> "log_entry_id: " >> escape >> log_entry_id >> "\n"; + in >> type_str >> "\n"; bool trailing_newline_found = false; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 3752c9deb8f..901669bfbdf 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -77,6 +77,7 @@ struct ReplicatedMergeTreeLogEntryData String toString() const; String znode_name; + String log_entry_id; Type type = EMPTY; String source_replica; /// Empty string means that this entry was added to the queue immediately, and not copied from the log. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 47d5a1bc895..335302e5dda 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5107,6 +5107,7 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry( LOG_DEBUG(log, "Waiting for {} to process log entry", replica); + if (startsWith(entry.znode_name, "log-")) { /// Take the number from the node name `log-xxxxxxxxxx`. UInt64 log_index = parse(entry.znode_name.substr(entry.znode_name.size() - 10)); @@ -5137,9 +5138,76 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry( if (!pulled_to_queue) return false; - } - LOG_DEBUG(log, "Looking for node corresponding to {} in {} queue", log_node_name, replica); + LOG_DEBUG(log, "Looking for node corresponding to {} in {} queue", log_node_name, replica); + } + else if (!entry.log_entry_id.empty()) + { + /// First pass, check the table log. + /// If found in the log, wait for replica to fetch it to the queue. + /// If not found in the log, it is already in the queue. + LOG_DEBUG(log, "Looking for log entry with id `{}` in the log", entry.log_entry_id); + + String log_pointer = getZooKeeper()->get(fs::path(table_zookeeper_path) / "replicas" / replica / "log_pointer"); + + Strings log_entries = getZooKeeper()->getChildren(fs::path(table_zookeeper_path) / "log"); + UInt64 log_index = 0; + bool found = false; + + for (const String & log_entry_name : log_entries) + { + log_index = parse(log_entry_name.substr(log_entry_name.size() - 10)); + + if (!log_pointer.empty() && log_index < parse(log_pointer)) + continue; + + String log_entry_str; + Coordination::Stat log_entry_stat; + bool exists = getZooKeeper()->tryGet(fs::path(table_zookeeper_path) / "log" / log_entry_name, log_entry_str, &log_entry_stat); + ReplicatedMergeTreeLogEntryData log_entry = *ReplicatedMergeTreeLogEntry::parse(log_entry_str, log_entry_stat); + if (exists && entry.log_entry_id == log_entry.log_entry_id) + { + LOG_DEBUG(log, "Found log entry with id `{}` in the log", entry.log_entry_id); + + found = true; + log_node_name = log_entry_name; + break; + } + } + + if (found) + { + LOG_DEBUG(log, "Waiting for {} to pull {} to queue", replica, log_node_name); + + /// Let's wait until entry gets into the replica queue. + bool pulled_to_queue = false; + do + { + zkutil::EventPtr event = std::make_shared(); + + log_pointer = getZooKeeper()->get(fs::path(table_zookeeper_path) / "replicas" / replica / "log_pointer", nullptr, event); + if (!log_pointer.empty() && parse(log_pointer) > log_index) + { + pulled_to_queue = true; + break; + } + + /// Wait with timeout because we can be already shut down, but not dropped. + /// So log_pointer node will exist, but we will never update it because all background threads already stopped. + /// It can lead to query hung because table drop query can wait for some query (alter, optimize, etc) which called this method, + /// but the query will never finish because the drop already shut down the table. + if (!stop_waiting()) + event->tryWait(event_wait_timeout_ms); + } while (!stop_waiting()); + + if (!pulled_to_queue) + return false; + } + } + else + { + throw Exception("Logical error: unexpected name of log node: " + entry.znode_name, ErrorCodes::LOGICAL_ERROR); + } /** Second - find the corresponding entry in the queue of the specified replica. * Its number may not match the `log` node. Therefore, we search by comparing the content. @@ -5151,12 +5219,25 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry( for (const String & entry_name : queue_entries) { String queue_entry_str; - bool exists = getZooKeeper()->tryGet(fs::path(table_zookeeper_path) / "replicas" / replica / "queue" / entry_name, queue_entry_str); + Coordination::Stat queue_entry_stat; + bool exists = getZooKeeper()->tryGet(fs::path(table_zookeeper_path) / "replicas" / replica / "queue" / entry_name, queue_entry_str, &queue_entry_stat); if (exists && queue_entry_str == entry_str) { queue_entry_to_wait_for = entry_name; break; } + else if (!entry.log_entry_id.empty()) + { + /// Check if the id matches rather than just contents. This entry + /// might have been written by different ClickHouse versions and + /// it is hard to guarantee same text representation. + ReplicatedMergeTreeLogEntryData queue_entry = *ReplicatedMergeTreeLogEntry::parse(queue_entry_str, queue_entry_stat); + if (entry.log_entry_id == queue_entry.log_entry_id) + { + queue_entry_to_wait_for = entry_name; + break; + } + } } /// While looking for the record, it has already been executed and deleted. From 9a02061d9c77fc0b6a252abf80f7615f2b3a1a13 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Thu, 16 Sep 2021 17:03:31 +0100 Subject: [PATCH 005/173] Rewrite part movement between shards logic and add kill support See https://github.com/ClickHouse/ClickHouse/pull/24585#issuecomment-857735081 for extra context about the current implementation. --- src/Access/AccessType.h | 2 +- .../InterpreterKillQueryQuery.cpp | 67 ++ src/Parsers/ASTKillQueryQuery.cpp | 16 +- src/Parsers/ASTKillQueryQuery.h | 1 + src/Parsers/ParserKillQueryQuery.cpp | 3 + src/Storages/IStorage.h | 6 + .../PartMovesBetweenShardsOrchestrator.cpp | 697 +++++++++++++----- .../PartMovesBetweenShardsOrchestrator.h | 53 +- src/Storages/StorageReplicatedMergeTree.cpp | 12 +- src/Storages/StorageReplicatedMergeTree.h | 3 + .../StorageSystemPartMovesBetweenShards.cpp | 6 +- .../test_part_moves_between_shards/test.py | 409 +++++++++- 12 files changed, 1014 insertions(+), 261 deletions(-) diff --git a/src/Access/AccessType.h b/src/Access/AccessType.h index 8e6a8be8e36..120a97d47b7 100644 --- a/src/Access/AccessType.h +++ b/src/Access/AccessType.h @@ -106,7 +106,7 @@ enum class AccessType (anyone can kill his own queries) */\ \ M(MOVE_PARTITION_BETWEEN_SHARDS, "", GLOBAL, ALL) /* required to be able to move a part/partition to a table - identified by it's ZooKeeper path */\ + identified by its ZooKeeper path */\ \ M(CREATE_USER, "", GLOBAL, ACCESS_MANAGEMENT) \ M(ALTER_USER, "", GLOBAL, ACCESS_MANAGEMENT) \ diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index d43d697fcd5..bfff1f1cd7b 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -31,6 +31,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int ACCESS_DENIED; + extern const int NOT_IMPLEMENTED; } @@ -290,6 +291,72 @@ BlockIO InterpreterKillQueryQuery::execute() break; } + case ASTKillQueryQuery::Type::PartMoveToShard: + { + if (query.sync) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "SYNC modifier is not supported for this statement."); + + Block moves_block = getSelectResult( + "database, table, task_name, task_uuid, part_name, to_shard, state", + "system.part_moves_between_shards"); + + if (!moves_block) + return res_io; + + const ColumnString & database_col = typeid_cast(*moves_block.getByName("database").column); + const ColumnString & table_col = typeid_cast(*moves_block.getByName("table").column); + const ColumnUUID & task_uuid_col = typeid_cast(*moves_block.getByName("task_uuid").column); + + auto header = moves_block.cloneEmpty(); + header.insert(0, {ColumnString::create(), std::make_shared(), "kill_status"}); + + MutableColumns res_columns = header.cloneEmptyColumns(); + auto table_id = StorageID::createEmpty(); + AccessRightsElements required_access_rights; + auto access = getContext()->getAccess(); + bool access_denied = false; + + for (size_t i = 0; i < moves_block.rows(); ++i) + { + table_id = StorageID{database_col.getDataAt(i).toString(), table_col.getDataAt(i).toString()}; + auto task_uuid = get(task_uuid_col[i]); + + CancellationCode code = CancellationCode::Unknown; + + if (!query.test) + { + auto storage = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); + if (!storage) + code = CancellationCode::NotFound; + else + { + ASTAlterCommand alter_command{}; + alter_command.type = ASTAlterCommand::MOVE_PARTITION; + alter_command.move_destination_type = DataDestinationType::SHARD; + required_access_rights = InterpreterAlterQuery::getRequiredAccessForCommand( + alter_command, table_id.database_name, table_id.table_name); + if (!access->isGranted(required_access_rights)) + { + access_denied = true; + continue; + } + code = storage->killPartMoveToShard(task_uuid); + } + } + + insertResultRow(i, code, moves_block, header, res_columns); + } + + if (res_columns[0]->empty() && access_denied) + throw Exception( + "Not allowed to kill move partition. To execute this query it's necessary to have the grant " + required_access_rights.toString(), + ErrorCodes::ACCESS_DENIED); + + + res_io.in = std::make_shared(header.cloneWithColumns(std::move(res_columns))); + + break; + } } return res_io; diff --git a/src/Parsers/ASTKillQueryQuery.cpp b/src/Parsers/ASTKillQueryQuery.cpp index 72bdd7d6b0b..71c3011dd2c 100644 --- a/src/Parsers/ASTKillQueryQuery.cpp +++ b/src/Parsers/ASTKillQueryQuery.cpp @@ -11,8 +11,20 @@ String ASTKillQueryQuery::getID(char delim) const void ASTKillQueryQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "KILL " - << (type == Type::Query ? "QUERY" : "MUTATION"); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "KILL "; + + switch (type) + { + case Type::Query: + settings.ostr << "QUERY"; + break; + case Type::Mutation: + settings.ostr << "MUTATION"; + break; + case Type::PartMoveToShard: + settings.ostr << "PART_MOVE_TO_SHARD"; + break; + } formatOnCluster(settings); diff --git a/src/Parsers/ASTKillQueryQuery.h b/src/Parsers/ASTKillQueryQuery.h index c1b3956962f..6ff12bcba93 100644 --- a/src/Parsers/ASTKillQueryQuery.h +++ b/src/Parsers/ASTKillQueryQuery.h @@ -13,6 +13,7 @@ public: { Query, /// KILL QUERY Mutation, /// KILL MUTATION + PartMoveToShard, /// KILL PART_MOVE_TO_SHARD }; Type type = Type::Query; diff --git a/src/Parsers/ParserKillQueryQuery.cpp b/src/Parsers/ParserKillQueryQuery.cpp index a195a778ed2..bc895406c9f 100644 --- a/src/Parsers/ParserKillQueryQuery.cpp +++ b/src/Parsers/ParserKillQueryQuery.cpp @@ -17,6 +17,7 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect ParserKeyword p_kill{"KILL"}; ParserKeyword p_query{"QUERY"}; ParserKeyword p_mutation{"MUTATION"}; + ParserKeyword p_part_move_to_shard{"PART_MOVE_TO_SHARD"}; ParserKeyword p_on{"ON"}; ParserKeyword p_test{"TEST"}; ParserKeyword p_sync{"SYNC"}; @@ -31,6 +32,8 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect query->type = ASTKillQueryQuery::Type::Query; else if (p_mutation.ignore(pos, expected)) query->type = ASTKillQueryQuery::Type::Mutation; + else if (p_part_move_to_shard.ignore(pos, expected)) + query->type = ASTKillQueryQuery::Type::PartMoveToShard; else return false; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 90cb963e064..9c340100469 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -460,6 +460,12 @@ public: throw Exception("Mutations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + /// Cancel a part move to shard. + virtual CancellationCode killPartMoveToShard(const UUID & /*task_uuid*/) + { + throw Exception("Part moves between shards are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + /** If the table have to do some complicated work on startup, * that must be postponed after creation of table object * (like launching some background threads), diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index c227febbbc2..b3a17250549 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -1,13 +1,20 @@ #include #include #include -#include #include #include #include namespace DB { + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; + extern const int TIMEOUT_EXCEEDED; +} + PartMovesBetweenShardsOrchestrator::PartMovesBetweenShardsOrchestrator(StorageReplicatedMergeTree & storage_) : storage(storage_) , zookeeper_path(storage.zookeeper_path) @@ -27,16 +34,17 @@ void PartMovesBetweenShardsOrchestrator::run() if (need_stop) return; - auto sleep_ms = 10; + /// Don't poll ZooKeeper too often. + auto sleep_ms = 3 * 1000; try { - fetchStateFromZK(); + syncStateFromZK(); + /// Schedule for immediate re-execution as likely there is more work + /// to be done. if (step()) - fetchStateFromZK(); - else - sleep_ms = 3 * 1000; + task->schedule(); } catch (...) { @@ -54,11 +62,11 @@ void PartMovesBetweenShardsOrchestrator::shutdown() LOG_TRACE(log, "PartMovesBetweenShardsOrchestrator thread finished"); } -void PartMovesBetweenShardsOrchestrator::fetchStateFromZK() +void PartMovesBetweenShardsOrchestrator::syncStateFromZK() { std::lock_guard lock(state_mutex); - entries.clear(); + std::vector new_entries; auto zk = storage.getZooKeeper(); @@ -76,8 +84,11 @@ void PartMovesBetweenShardsOrchestrator::fetchStateFromZK() e.version = stat.version; e.znode_name = task_name; - entries[task_name] = std::move(e); + new_entries.push_back(std::move(e)); } + + // Replace in-memory state. + entries = new_entries; } bool PartMovesBetweenShardsOrchestrator::step() @@ -93,7 +104,7 @@ bool PartMovesBetweenShardsOrchestrator::step() { std::lock_guard lock(state_mutex); - for (auto const & entry : entries | boost::adaptors::map_values) + for (auto const & entry : entries) { if (entry.state.value == EntryState::DONE || entry.state.value == EntryState::CANCELLED) continue; @@ -128,11 +139,21 @@ bool PartMovesBetweenShardsOrchestrator::step() throw; } + LOG_DEBUG(log, "stepEntry on task {} from state {} (rollback: {}), try: {}", + entry_to_process->znode_name, + entry_to_process->state.toString(), + entry_to_process->rollback, + entry_to_process->num_tries); + try { /// Use the same ZooKeeper connection. If we'd lost the lock then connection /// will become expired and all consequent operations will fail. - stepEntry(entry_to_process.value(), zk); + Entry new_entry = stepEntry(entry_to_process.value(), zk); + new_entry.last_exception_msg = ""; + new_entry.num_tries = 0; + new_entry.update_time = std::time(nullptr); + zk->set(new_entry.znode_path, new_entry.toString(), new_entry.version); } catch (...) { @@ -140,6 +161,7 @@ bool PartMovesBetweenShardsOrchestrator::step() Entry entry_copy = entry_to_process.value(); entry_copy.last_exception_msg = getCurrentExceptionMessage(false); + entry_copy.num_tries += 1; entry_copy.update_time = std::time(nullptr); zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); @@ -149,276 +171,537 @@ bool PartMovesBetweenShardsOrchestrator::step() return true; } -void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry, zkutil::ZooKeeperPtr zk) +PartMovesBetweenShardsOrchestrator::Entry PartMovesBetweenShardsOrchestrator::stepEntry(Entry entry, zkutil::ZooKeeperPtr zk) { switch (entry.state.value) { - case EntryState::DONE: - break; - + case EntryState::DONE: [[fallthrough]]; case EntryState::CANCELLED: - break; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't stepEntry after terminal state. This is a bug."); case EntryState::TODO: { - /// State transition. - Entry entry_copy = entry; - entry_copy.state = EntryState::SYNC_SOURCE; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + if (entry.rollback) + { + removePins(entry, zk); + entry.state = EntryState::CANCELLED; + return entry; + } + /// The forward transition happens implicitly when task is created by `StorageReplicatedMergeTree::movePartitionToShard`. + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected entry state ({}) in stepEntry. This is a bug.", entry.state.toString()); } - break; case EntryState::SYNC_SOURCE: { + if (entry.rollback) { - /// Log entry. - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); - - ReplicatedMergeTreeLogEntryData log_entry; - log_entry.type = ReplicatedMergeTreeLogEntryData::SYNC_PINNED_PART_UUIDS; - log_entry.create_time = std::time(nullptr); - log_entry.source_replica = storage.replica_name; - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); - ops.emplace_back(zkutil::makeCreateRequest( - zookeeper_path + "/log/log-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); - - Coordination::Responses responses; - Coordination::Error rc = zk->tryMulti(ops, responses); - zkutil::KeeperMultiException::check(rc, ops, responses); - - String log_znode_path = dynamic_cast(*responses.back()).path_created; - log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); - - /// This wait in background schedule pool is useless. It'd be - /// better to have some notification which will call `step` - /// function when all replicated will finish. TODO. - storage.waitForAllReplicasToProcessLogEntry(zookeeper_path, log_entry, -1); + entry.state = EntryState::TODO; + return entry; } - + else { - /// State transition. - Entry entry_copy = entry; - entry_copy.state = EntryState::SYNC_DESTINATION; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + ReplicatedMergeTreeLogEntryData sync_source_log_entry; + + String sync_source_log_entry_barrier_path = fs::path(entry.znode_path) / ("log_" + entry.state.toString()); + Coordination::Stat sync_source_log_entry_stat; + String sync_source_log_entry_str; + if (zk->tryGet(sync_source_log_entry_barrier_path, sync_source_log_entry_str, &sync_source_log_entry_stat)) + { + LOG_DEBUG(log, "Log entry was already created will check the existing one."); + + sync_source_log_entry = *ReplicatedMergeTreeLogEntry::parse(sync_source_log_entry_str, sync_source_log_entry_stat); + } + else + { + /// Log entry. + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); + + sync_source_log_entry.type = ReplicatedMergeTreeLogEntryData::SYNC_PINNED_PART_UUIDS; + sync_source_log_entry.log_entry_id = sync_source_log_entry_barrier_path; + sync_source_log_entry.create_time = std::time(nullptr); + sync_source_log_entry.source_replica = storage.replica_name; + + ops.emplace_back(zkutil::makeCreateRequest(sync_source_log_entry_barrier_path, sync_source_log_entry.toString(), -1)); + ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest( + zookeeper_path + "/log/log-", sync_source_log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + + Coordination::Responses responses; + Coordination::Error rc = zk->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path = dynamic_cast(*responses.back()).path_created; + sync_source_log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + LOG_DEBUG(log, "Pushed log entry: {}", log_znode_path); + } + + Strings unwaited = storage.tryWaitForAllReplicasToProcessLogEntry(zookeeper_path, sync_source_log_entry, 1); + if (!unwaited.empty()) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Some replicas haven't processed event: {}, will retry later.", toString(unwaited)); + + entry.state = EntryState::SYNC_DESTINATION; + return entry; } } - break; case EntryState::SYNC_DESTINATION: { + if (entry.rollback) { - /// Log entry. - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); - - ReplicatedMergeTreeLogEntryData log_entry; - log_entry.type = ReplicatedMergeTreeLogEntryData::SYNC_PINNED_PART_UUIDS; - log_entry.create_time = std::time(nullptr); - log_entry.source_replica = storage.replica_name; - log_entry.source_shard = zookeeper_path; - - ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); - ops.emplace_back(zkutil::makeCreateRequest( - entry.to_shard + "/log/log-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); - - Coordination::Responses responses; - Coordination::Error rc = zk->tryMulti(ops, responses); - zkutil::KeeperMultiException::check(rc, ops, responses); - - String log_znode_path = dynamic_cast(*responses.back()).path_created; - log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); - - storage.waitForAllReplicasToProcessLogEntry(entry.to_shard, log_entry, -1); - } - - { - /// State transition. Entry entry_copy = entry; - entry_copy.state = EntryState::DESTINATION_FETCH; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + entry_copy.state = EntryState::SYNC_SOURCE; + return entry_copy; + } + else + { + ReplicatedMergeTreeLogEntryData sync_destination_log_entry; + + String sync_destination_log_entry_barrier_path = fs::path(entry.znode_path) / ("log_" + entry.state.toString()); + Coordination::Stat sync_destination_log_entry_stat; + String sync_destination_log_entry_str; + if (zk->tryGet(sync_destination_log_entry_barrier_path, sync_destination_log_entry_str, &sync_destination_log_entry_stat)) + { + LOG_DEBUG(log, "Log entry was already created will check the existing one."); + + sync_destination_log_entry = *ReplicatedMergeTreeLogEntry::parse(sync_destination_log_entry_str, sync_destination_log_entry_stat); + } + else + { + /// Log entry. + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); + + sync_destination_log_entry.type = ReplicatedMergeTreeLogEntryData::SYNC_PINNED_PART_UUIDS; + sync_destination_log_entry.log_entry_id = sync_destination_log_entry_barrier_path; + sync_destination_log_entry.create_time = std::time(nullptr); + sync_destination_log_entry.source_replica = storage.replica_name; + sync_destination_log_entry.source_shard = zookeeper_path; + + ops.emplace_back(zkutil::makeCreateRequest(sync_destination_log_entry_barrier_path, sync_destination_log_entry.toString(), -1)); + ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest( + entry.to_shard + "/log/log-", sync_destination_log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + + Coordination::Responses responses; + Coordination::Error rc = zk->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path = dynamic_cast(*responses.back()).path_created; + sync_destination_log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + LOG_DEBUG(log, "Pushed log entry: {}", log_znode_path); + } + + Strings unwaited = storage.tryWaitForAllReplicasToProcessLogEntry(entry.to_shard, sync_destination_log_entry, 1); + if (!unwaited.empty()) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Some replicas haven't processed event: {}, will retry later.", toString(unwaited)); + + entry.state = EntryState::DESTINATION_FETCH; + return entry; } } - break; case EntryState::DESTINATION_FETCH: { - /// Make sure table structure doesn't change when there are part movements in progress. + + if (entry.rollback) { - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); + // TODO(nv): Do we want to cleanup fetched data on the destination? + // Maybe leave it there and make sure a background cleanup will take + // care of it sometime later. - /// Log entry. - ReplicatedMergeTreeLogEntryData log_entry; - log_entry.type = ReplicatedMergeTreeLogEntryData::CLONE_PART_FROM_SHARD; - log_entry.create_time = std::time(nullptr); - log_entry.new_part_name = entry.part_name; - log_entry.source_replica = storage.replica_name; - log_entry.source_shard = zookeeper_path; - ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); - ops.emplace_back(zkutil::makeCreateRequest( - entry.to_shard + "/log/log-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); - - Coordination::Responses responses; - Coordination::Error rc = zk->tryMulti(ops, responses); - zkutil::KeeperMultiException::check(rc, ops, responses); - - String log_znode_path = dynamic_cast(*responses.back()).path_created; - log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); - - storage.waitForAllReplicasToProcessLogEntry(entry.to_shard, log_entry, -1); + entry.state = EntryState::SYNC_DESTINATION; + return entry; } - + else { - /// State transition. - Entry entry_copy = entry; - entry_copy.state = EntryState::DESTINATION_ATTACH; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + /// Note: Table structure shouldn't be changed while there are part movements in progress. + + ReplicatedMergeTreeLogEntryData fetch_log_entry; + + String fetch_log_entry_barrier_path = fs::path(entry.znode_path) / ("log_" + entry.state.toString()); + Coordination::Stat fetch_log_entry_stat; + String fetch_log_entry_str; + if (zk->tryGet(fetch_log_entry_barrier_path, fetch_log_entry_str, &fetch_log_entry_stat)) + { + LOG_DEBUG(log, "Log entry was already created will check the existing one."); + + fetch_log_entry = *ReplicatedMergeTreeLogEntry::parse(fetch_log_entry_str, fetch_log_entry_stat); + } + else + { + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); + + fetch_log_entry.type = ReplicatedMergeTreeLogEntryData::CLONE_PART_FROM_SHARD; + fetch_log_entry.log_entry_id = fetch_log_entry_barrier_path; + fetch_log_entry.create_time = std::time(nullptr); + fetch_log_entry.new_part_name = entry.part_name; + fetch_log_entry.source_replica = storage.replica_name; + fetch_log_entry.source_shard = zookeeper_path; + ops.emplace_back(zkutil::makeCreateRequest(fetch_log_entry_barrier_path, fetch_log_entry.toString(), -1)); + ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest( + entry.to_shard + "/log/log-", fetch_log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + + Coordination::Responses responses; + Coordination::Error rc = zk->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path = dynamic_cast(*responses.back()).path_created; + fetch_log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + LOG_DEBUG(log, "Pushed log entry: {}", log_znode_path); + } + + Strings unwaited = storage.tryWaitForAllReplicasToProcessLogEntry(entry.to_shard, fetch_log_entry, 1); + if (!unwaited.empty()) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Some replicas haven't processed event: {}, will retry later.", toString(unwaited)); + + entry.state = EntryState::DESTINATION_ATTACH; + return entry; } } - break; case EntryState::DESTINATION_ATTACH: { - /// There is a chance that attach on destination will fail and this task will be left in the queue forever. + String attach_log_entry_barrier_path = fs::path(entry.znode_path) / ("log_" + entry.state.toString()); + + if (entry.rollback) { + Coordination::Stat attach_log_entry_stat; + String attach_log_entry_str; + if (!zk->tryGet(attach_log_entry_barrier_path, attach_log_entry_str, &attach_log_entry_stat)) + { + LOG_DEBUG(log, "Log entry for DESTINATION_ATTACH not found. Not sending DROP_RANGE log entry."); + + // ATTACH_PART wasn't issued, nothing to revert. + entry.state = EntryState::DESTINATION_FETCH; + return entry; + } + else + { + // Need to remove ATTACH_PART from the queue or drop data. + // Similar to `StorageReplicatedMergeTree::dropPart` w/o extra + // checks as we know drop shall be possible. + ReplicatedMergeTreeLogEntryData attach_rollback_log_entry; + + String attach_rollback_log_entry_barrier_path = fs::path(entry.znode_path) / ("log_" + entry.state.toString() + "_rollback"); + Coordination::Stat attach_rollback_log_entry_stat; + String attach_rollback_log_entry_str; + if (zk->tryGet(attach_rollback_log_entry_barrier_path, attach_rollback_log_entry_str, &attach_rollback_log_entry_stat)) + { + LOG_DEBUG(log, "Log entry was already created will check the existing one."); + + attach_rollback_log_entry = *ReplicatedMergeTreeLogEntry::parse(attach_rollback_log_entry_str, attach_rollback_log_entry_stat); + } + else + { + const auto attach_log_entry = ReplicatedMergeTreeLogEntry::parse(attach_log_entry_str, attach_log_entry_stat); + + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); + + auto drop_part_info = MergeTreePartInfo::fromPartName(attach_log_entry->new_part_name, storage.format_version); + + storage.getClearBlocksInPartitionOps( + ops, *zk, drop_part_info.partition_id, drop_part_info.min_block, drop_part_info.max_block); + size_t clear_block_ops_size = ops.size(); + + attach_rollback_log_entry.type = ReplicatedMergeTreeLogEntryData::DROP_RANGE; + attach_rollback_log_entry.log_entry_id = attach_rollback_log_entry_barrier_path; + attach_rollback_log_entry.source_replica = storage.replica_name; + attach_rollback_log_entry.source_shard = zookeeper_path; + + attach_rollback_log_entry.new_part_name = getPartNamePossiblyFake(storage.format_version, drop_part_info); + attach_rollback_log_entry.create_time = time(nullptr); + + ops.emplace_back(zkutil::makeCreateRequest(attach_rollback_log_entry_barrier_path, attach_rollback_log_entry.toString(), -1)); + ops.emplace_back(zkutil::makeCreateRequest( + entry.to_shard + "/log/log-", attach_rollback_log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); + Coordination::Responses responses; + Coordination::Error rc = zk->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path + = dynamic_cast(*responses[clear_block_ops_size]).path_created; + attach_rollback_log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + LOG_DEBUG(log, "Pushed log entry: {}", log_znode_path); + } + + Strings unwaited = storage.tryWaitForAllReplicasToProcessLogEntry(entry.to_shard, attach_rollback_log_entry, 1); + if (!unwaited.empty()) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Some replicas haven't processed event: {}, will retry later.", toString(unwaited)); + + entry.state = EntryState::DESTINATION_FETCH; + return entry; + } + } + else + { + /// There is a chance that attach on destination will fail and this task will be left in the queue forever. + Coordination::Requests ops; ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); auto part = storage.getActiveContainingPart(entry.part_name); + /// Allocating block number in other replicas zookeeper path /// TODO Maybe we can do better. - auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zk, "", entry.to_shard); - auto block_number = block_number_lock->getNumber(); + auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zk, attach_log_entry_barrier_path, entry.to_shard); - auto part_info = part->info; - part_info.min_block = block_number; - part_info.max_block = block_number; - part_info.level = 0; - part_info.mutation = 0; - - /// Attach log entry (all replicas already fetched part) ReplicatedMergeTreeLogEntryData log_entry; - log_entry.type = ReplicatedMergeTreeLogEntryData::ATTACH_PART; - log_entry.part_checksum = part->checksums.getTotalChecksumHex(); - log_entry.create_time = std::time(nullptr); - log_entry.new_part_name = part_info.getPartName(); - ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); - ops.emplace_back(zkutil::makeCreateRequest( - entry.to_shard + "/log/log-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); - Coordination::Responses responses; - Coordination::Error rc = zk->tryMulti(ops, responses); - zkutil::KeeperMultiException::check(rc, ops, responses); + if (block_number_lock) + { + auto block_number = block_number_lock->getNumber(); - String log_znode_path = dynamic_cast(*responses.back()).path_created; - log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + auto part_info = part->info; + part_info.min_block = block_number; + part_info.max_block = block_number; + part_info.level = 0; + part_info.mutation = 0; - storage.waitForAllReplicasToProcessLogEntry(entry.to_shard, log_entry, -1); - } + /// Attach log entry (all replicas already fetched part) + log_entry.type = ReplicatedMergeTreeLogEntryData::ATTACH_PART; + log_entry.log_entry_id = attach_log_entry_barrier_path; + log_entry.part_checksum = part->checksums.getTotalChecksumHex(); + log_entry.create_time = std::time(nullptr); + log_entry.new_part_name = part_info.getPartName(); - { - /// State transition. - Entry entry_copy = entry; - entry_copy.state = EntryState::SOURCE_DROP_PRE_DELAY; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + ops.emplace_back(zkutil::makeCreateRequest(attach_log_entry_barrier_path, log_entry.toString(), -1)); + ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest( + entry.to_shard + "/log/log-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + + Coordination::Responses responses; + Coordination::Error rc = zk->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path = dynamic_cast(*responses.back()).path_created; + log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + LOG_DEBUG(log, "Pushed log entry: {}", log_znode_path); + } + else + { + LOG_DEBUG(log, "Log entry was already created will check the existing one."); + + Coordination::Stat stat; + String log_entry_str = zk->get(attach_log_entry_barrier_path, &stat); + log_entry = *ReplicatedMergeTreeLogEntry::parse(log_entry_str, stat); + } + + Strings unwaited = storage.tryWaitForAllReplicasToProcessLogEntry(entry.to_shard, log_entry, 1); + if (!unwaited.empty()) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Some replicas haven't processed event: {}, will retry later.", toString(unwaited)); + + + entry.dst_part_name = log_entry.new_part_name; + entry.state = EntryState::SOURCE_DROP_PRE_DELAY; + return entry; } } - break; case EntryState::SOURCE_DROP_PRE_DELAY: { - std::this_thread::sleep_for(std::chrono::seconds(storage.getSettings()->part_moves_between_shards_delay_seconds)); - - /// State transition. - Entry entry_copy = entry; - entry_copy.state = EntryState::SOURCE_DROP; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + if (entry.rollback) + { + entry.state = EntryState::DESTINATION_ATTACH; + return entry; + } + else + { + std::this_thread::sleep_for(std::chrono::seconds(storage.getSettings()->part_moves_between_shards_delay_seconds)); + entry.state = EntryState::SOURCE_DROP; + return entry; + } } - break; case EntryState::SOURCE_DROP: { + if (entry.rollback) + throw Exception(ErrorCodes::LOGICAL_ERROR, "It is not possible to rollback from this state. This is a bug."); + else { - ReplicatedMergeTreeLogEntry log_entry; - if (storage.dropPartImpl(zk, entry.part_name, log_entry, false, false)) - storage.waitForAllReplicasToProcessLogEntry(zookeeper_path, log_entry, -1); - } + // Can't use dropPartImpl directly as we need additional zk ops to remember the log entry + // for subsequent retries. - { - /// State transition. - Entry entry_copy = entry; - entry_copy.state = EntryState::SOURCE_DROP_POST_DELAY; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + ReplicatedMergeTreeLogEntryData source_drop_log_entry; + + String source_drop_log_entry_barrier_path = fs::path(entry.znode_path) / ("log_" + entry.state.toString()); + Coordination::Stat source_drop_log_entry_stat; + String source_drop_log_entry_str; + if (zk->tryGet(source_drop_log_entry_barrier_path, source_drop_log_entry_str, &source_drop_log_entry_stat)) + { + LOG_DEBUG(log, "Log entry was already created will check the existing one."); + + source_drop_log_entry = *ReplicatedMergeTreeLogEntry::parse(source_drop_log_entry_str, source_drop_log_entry_stat); + } + else + { + auto source_drop_part_info = MergeTreePartInfo::fromPartName(entry.part_name, storage.format_version); + + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); + + storage.getClearBlocksInPartitionOps(ops, *zk, source_drop_part_info.partition_id, source_drop_part_info.min_block, source_drop_part_info.max_block); + + source_drop_log_entry.type = ReplicatedMergeTreeLogEntryData::DROP_RANGE; + source_drop_log_entry.log_entry_id = source_drop_log_entry_barrier_path; + source_drop_log_entry.create_time = std::time(nullptr); + source_drop_log_entry.new_part_name = getPartNamePossiblyFake(storage.format_version, source_drop_part_info); + source_drop_log_entry.source_replica = storage.replica_name; + + ops.emplace_back(zkutil::makeCreateRequest(source_drop_log_entry_barrier_path, source_drop_log_entry.toString(), -1)); + ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest( + zookeeper_path + "/log/log-", source_drop_log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + + Coordination::Responses responses; + Coordination::Error rc = zk->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path = dynamic_cast(*responses.back()).path_created; + source_drop_log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + LOG_DEBUG(log, "Pushed log entry: {}", log_znode_path); + } + + Strings unwaited = storage.tryWaitForAllReplicasToProcessLogEntry(zookeeper_path, source_drop_log_entry, 1); + if (!unwaited.empty()) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Some replicas haven't processed event: {}, will retry later.", toString(unwaited)); + + entry.state = EntryState::SOURCE_DROP_POST_DELAY; + return entry; } } - break; case EntryState::SOURCE_DROP_POST_DELAY: { - std::this_thread::sleep_for(std::chrono::seconds(storage.getSettings()->part_moves_between_shards_delay_seconds)); - - /// State transition. - Entry entry_copy = entry; - entry_copy.state = EntryState::REMOVE_UUID_PIN; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + if (entry.rollback) + throw Exception(ErrorCodes::LOGICAL_ERROR, "It is not possible to rollback from this state. This is a bug."); + else + { + std::this_thread::sleep_for(std::chrono::seconds(storage.getSettings()->part_moves_between_shards_delay_seconds)); + entry.state = EntryState::REMOVE_UUID_PIN; + return entry; + } } - break; case EntryState::REMOVE_UUID_PIN: { + if (entry.rollback) + throw Exception(ErrorCodes::LOGICAL_ERROR, "It is not possible to rollback from this state. This is a bug."); + else { - PinnedPartUUIDs src_pins; - PinnedPartUUIDs dst_pins; + removePins(entry, zk); - { - String s = zk->get(zookeeper_path + "/pinned_part_uuids", &src_pins.stat); - src_pins.fromString(s); - } - - { - String s = zk->get(entry.to_shard + "/pinned_part_uuids", &dst_pins.stat); - dst_pins.fromString(s); - } - - src_pins.part_uuids.erase(entry.part_uuid); - dst_pins.part_uuids.erase(entry.part_uuid); - - Coordination::Requests ops; - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/pinned_part_uuids", src_pins.toString(), src_pins.stat.version)); - ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/pinned_part_uuids", dst_pins.toString(), dst_pins.stat.version)); - - zk->multi(ops); + entry.state = EntryState::DONE; + return entry; } - - /// State transition. - Entry entry_copy = entry; - entry_copy.state = EntryState::DONE; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); } - break; + } + + __builtin_unreachable(); +} + +void PartMovesBetweenShardsOrchestrator::removePins(const Entry & entry, zkutil::ZooKeeperPtr zk) +{ + PinnedPartUUIDs src_pins; + PinnedPartUUIDs dst_pins; + + { + String s = zk->get(zookeeper_path + "/pinned_part_uuids", &src_pins.stat); + src_pins.fromString(s); + } + + { + String s = zk->get(entry.to_shard + "/pinned_part_uuids", &dst_pins.stat); + dst_pins.fromString(s); + } + + dst_pins.part_uuids.erase(entry.part_uuid); + src_pins.part_uuids.erase(entry.part_uuid); + + Coordination::Requests ops; + ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/pinned_part_uuids", src_pins.toString(), src_pins.stat.version)); + ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/pinned_part_uuids", dst_pins.toString(), dst_pins.stat.version)); + + zk->multi(ops); +} + +CancellationCode PartMovesBetweenShardsOrchestrator::killPartMoveToShard(const UUID & task_uuid) +{ + while (true) + { + auto entry = getEntryByUUID(task_uuid); + + // If the task is in this state or any that follows it is too late to rollback + // since we can't be sure if the source data still exists. + auto not_possible_to_rollback_after_state = EntryState(EntryState::SOURCE_DROP); + if (entry.state.value >= not_possible_to_rollback_after_state.value) + { + LOG_DEBUG(log, "Can't kill move part between shards entry {} ({}) after state {}. Current state: {}.", + toString(entry.task_uuid), entry.znode_name, not_possible_to_rollback_after_state.toString(), entry.state.toString()); + return CancellationCode::CancelCannotBeSent; + } + + LOG_TRACE(log, "Will try to mark move part between shards entry {} ({}) for rollback.", + toString(entry.task_uuid), entry.znode_name); + + auto zk = storage.getZooKeeper(); + + // State transition. + entry.rollback = true; + entry.update_time = std::time(nullptr); + entry.num_tries = 0; + entry.last_exception_msg = ""; + + auto code = zk->trySet(entry.znode_path, entry.toString(), entry.version); + if (code == Coordination::Error::ZOK) + { + // Orchestrator will process it in background. + return CancellationCode::CancelSent; + } + 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, entry.znode_path); } } -std::vector PartMovesBetweenShardsOrchestrator::getEntries() const +std::vector PartMovesBetweenShardsOrchestrator::getEntries() { + // Force sync. Also catches parsing errors. + syncStateFromZK(); + std::lock_guard lock(state_mutex); - std::vector res; + return entries; +} - for (const auto & e : entries) - res.push_back(e.second); +PartMovesBetweenShardsOrchestrator::Entry PartMovesBetweenShardsOrchestrator::getEntryByUUID(const UUID & task_uuid) +{ + /// Need latest state in case user tries to kill a move observed on a different replica. + syncStateFromZK(); - return res; + std::lock_guard lock(state_mutex); + for (auto const & entry : entries) + { + if (entry.task_uuid == task_uuid) + return entry; + } + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Task with id {} not found", toString(task_uuid)); } String PartMovesBetweenShardsOrchestrator::Entry::toString() const @@ -431,12 +714,19 @@ String PartMovesBetweenShardsOrchestrator::Entry::toString() const json.set(JSON_KEY_PART_NAME, part_name); json.set(JSON_KEY_PART_UUID, DB::toString(part_uuid)); json.set(JSON_KEY_TO_SHARD, to_shard); + json.set(JSON_KEY_DST_PART_NAME, dst_part_name); json.set(JSON_KEY_STATE, state.toString()); + json.set(JSON_KEY_ROLLBACK, DB::toString(rollback)); json.set(JSON_KEY_LAST_EX_MSG, last_exception_msg); + json.set(JSON_KEY_NUM_TRIES, DB::toString(num_tries)); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); - json.stringify(oss); + + // Always escape unicode to make last_exception_msg json safe. + // It may contain random binary data when exception is a parsing error + // of unexpected contents. + Poco::JSON::Stringifier::stringify(json, oss, 0, -1, Poco::JSON_WRAP_STRINGS | Poco::JSON_ESCAPE_UNICODE); return oss.str(); } @@ -452,8 +742,11 @@ void PartMovesBetweenShardsOrchestrator::Entry::fromString(const String & buf) part_name = json->getValue(JSON_KEY_PART_NAME); part_uuid = parseFromString(json->getValue(JSON_KEY_PART_UUID)); to_shard = json->getValue(JSON_KEY_TO_SHARD); + dst_part_name = json->getValue(JSON_KEY_DST_PART_NAME); state.value = EntryState::fromString(json->getValue(JSON_KEY_STATE)); + rollback = json->getValue(JSON_KEY_ROLLBACK); last_exception_msg = json->getValue(JSON_KEY_LAST_EX_MSG); + num_tries = json->getValue(JSON_KEY_NUM_TRIES); } } diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h index 9e54ae8a8ed..38b6a076748 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -18,14 +19,37 @@ namespace ErrorCodes class StorageReplicatedMergeTree; -/// Cross shard part movement workflow orchestration. +/** + * Cross shard part movement workflow orchestration. + * + * TODO(nv): + * * Usage of `format_version` when acting on the behalf of the remote shard. + * There needs to be sort of an API to coordinate with remote replicas. + * * Only one movement at a time can be coordinated. This can easily be fixed + * by cycling through different tasks and checking their status with a + * priority queue and back-off for failing tasks + * `min(backoff * num_tries, max_backoff)`. + */ class PartMovesBetweenShardsOrchestrator { public: struct EntryState { + // State transitions are linear. When a kill query is issued a rollback + // flag is set and transitions order is reversed. + // + // SOURCE_DROP is a critical state after which rollback is not possible + // and we must ensure that the task can always succeed after that. + // + // Similar for rollback. It should be always possible to rollback before + // SOURCE_DROP state and it should terminate. + // + // Note: This fragile. If you change the states please add entry to + // changelog about forward/backward compatibility. Better not to have + // any active move tasks while doing upgrade/downgrade operations. enum Value { + CANCELLED, TODO, SYNC_SOURCE, SYNC_DESTINATION, @@ -36,7 +60,6 @@ public: SOURCE_DROP_POST_DELAY, REMOVE_UUID_PIN, DONE, - CANCELLED, }; EntryState(): value(TODO) {} @@ -95,10 +118,14 @@ public: String part_name; UUID part_uuid; String to_shard; + String dst_part_name; EntryState state; + bool rollback = false; + /// Reset on successful transitions. String last_exception_msg; + UInt64 num_tries = 0; String znode_name; @@ -120,27 +147,31 @@ private: static constexpr auto JSON_KEY_PART_NAME = "part_name"; static constexpr auto JSON_KEY_PART_UUID = "part_uuid"; static constexpr auto JSON_KEY_TO_SHARD = "to_shard"; + static constexpr auto JSON_KEY_DST_PART_NAME = "dst_part_name"; static constexpr auto JSON_KEY_STATE = "state"; + static constexpr auto JSON_KEY_ROLLBACK = "rollback"; static constexpr auto JSON_KEY_LAST_EX_MSG = "last_exception"; + static constexpr auto JSON_KEY_NUM_TRIES = "num_tries"; public: - PartMovesBetweenShardsOrchestrator(StorageReplicatedMergeTree & storage_); + explicit PartMovesBetweenShardsOrchestrator(StorageReplicatedMergeTree & storage_); void start() { task->activateAndSchedule(); } void wakeup() { task->schedule(); } void shutdown(); - void fetchStateFromZK(); + CancellationCode killPartMoveToShard(const UUID & task_uuid); - /// We could have one thread per Entry and worry about concurrency issues. - /// Or we could have a single thread trying to run one step at a time. - bool step(); - - std::vector getEntries() const; + std::vector getEntries(); private: void run(); - void stepEntry(const Entry & entry, zkutil::ZooKeeperPtr zk); + bool step(); + Entry stepEntry(Entry entry, zkutil::ZooKeeperPtr zk); + + Entry getEntryByUUID(const UUID & task_uuid); + void removePins(const Entry & entry, zkutil::ZooKeeperPtr zk); + void syncStateFromZK(); private: StorageReplicatedMergeTree & storage; @@ -153,7 +184,7 @@ private: BackgroundSchedulePool::TaskHolder task; mutable std::mutex state_mutex; - std::map entries; + std::vector entries; public: String entries_znode_path; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 335302e5dda..03bef4581c1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4669,7 +4669,7 @@ void StorageReplicatedMergeTree::alter( /// If new version returns ordinary name, else returns part name containing the first and last month of the month /// NOTE: use it in pair with getFakePartCoveringAllPartsInPartition(...) -static String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info) +String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info) { if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { @@ -6568,6 +6568,7 @@ void StorageReplicatedMergeTree::movePartitionToShard( dst_pins.part_uuids.insert(part->uuid); PartMovesBetweenShardsOrchestrator::Entry part_move_entry; + part_move_entry.state = PartMovesBetweenShardsOrchestrator::EntryState::SYNC_SOURCE; part_move_entry.create_time = std::time(nullptr); part_move_entry.update_time = part_move_entry.create_time; part_move_entry.task_uuid = UUIDHelpers::generateV4(); @@ -6591,10 +6592,13 @@ void StorageReplicatedMergeTree::movePartitionToShard( String task_znode_path = dynamic_cast(*responses.back()).path_created; LOG_DEBUG(log, "Created task for part movement between shards at " + task_znode_path); - /// Force refresh local state. This will make the task immediately visible in `system.part_moves_between_shards` table. - part_moves_between_shards_orchestrator.fetchStateFromZK(); + /// TODO(nv): Nice to have support for `replication_alter_partitions_sync`. + /// For now use the system.part_moves_between_shards table for status. +} - // TODO: Add support for `replication_alter_partitions_sync`. +CancellationCode StorageReplicatedMergeTree::killPartMoveToShard(const UUID & task_uuid) +{ + return part_moves_between_shards_orchestrator.killPartMoveToShard(task_uuid); } void StorageReplicatedMergeTree::getCommitPartOps( diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index e76a54ac5ec..9eaf3c318dd 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -682,6 +682,7 @@ private: void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr query_context) override; void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context) override; void movePartitionToShard(const ASTPtr & partition, bool move_part, const String & to, ContextPtr query_context) override; + CancellationCode killPartMoveToShard(const UUID & task_uuid) override; void fetchPartition( const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, @@ -745,6 +746,8 @@ protected: bool allow_renaming_); }; +String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info); + /** There are three places for each part, where it should be * 1. In the RAM, data_parts, all_data_parts. diff --git a/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp b/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp index 70bc473e241..c397392e9fb 100644 --- a/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp +++ b/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp @@ -30,10 +30,12 @@ NamesAndTypesList StorageSystemPartMovesBetweenShards::getNamesAndTypes() { "part_name", std::make_shared() }, { "part_uuid", std::make_shared() }, { "to_shard", std::make_shared() }, + { "dst_part_name", std::make_shared() }, /// Processing status of item. { "update_time", std::make_shared() }, { "state", std::make_shared() }, + { "rollback", std::make_shared() }, { "num_tries", std::make_shared() }, { "last_exception", std::make_shared() }, }; @@ -122,11 +124,13 @@ void StorageSystemPartMovesBetweenShards::fillData(MutableColumns & res_columns, res_columns[col_num++]->insert(entry.part_name); res_columns[col_num++]->insert(entry.part_uuid); res_columns[col_num++]->insert(entry.to_shard); + res_columns[col_num++]->insert(entry.dst_part_name); /// Processing status of item. res_columns[col_num++]->insert(entry.update_time); res_columns[col_num++]->insert(entry.state.toString()); - res_columns[col_num++]->insert(0); + res_columns[col_num++]->insert(entry.rollback); + res_columns[col_num++]->insert(entry.num_tries); res_columns[col_num++]->insert(entry.last_exception_msg); } } diff --git a/tests/integration/test_part_moves_between_shards/test.py b/tests/integration/test_part_moves_between_shards/test.py index 00407f95389..ed7640e5f9e 100644 --- a/tests/integration/test_part_moves_between_shards/test.py +++ b/tests/integration/test_part_moves_between_shards/test.py @@ -1,32 +1,38 @@ -import random -import time - import pytest +import random +import threading +import time from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV +transient_ch_errors = [23, 32, 210] + cluster = ClickHouseCluster(__file__) s0r0 = cluster.add_instance( 's0r0', main_configs=['configs/remote_servers.xml', 'configs/merge_tree.xml'], + stay_alive=True, with_zookeeper=True) s0r1 = cluster.add_instance( 's0r1', main_configs=['configs/remote_servers.xml', 'configs/merge_tree.xml'], + stay_alive=True, with_zookeeper=True) s1r0 = cluster.add_instance( 's1r0', main_configs=['configs/remote_servers.xml', 'configs/merge_tree.xml'], + stay_alive=True, with_zookeeper=True) s1r1 = cluster.add_instance( 's1r1', main_configs=['configs/remote_servers.xml', 'configs/merge_tree.xml'], + stay_alive=True, with_zookeeper=True) @@ -43,12 +49,14 @@ def test_move(started_cluster): for shard_ix, rs in enumerate([[s0r0, s0r1], [s1r0, s1r1]]): for replica_ix, r in enumerate(rs): r.query(""" + DROP TABLE IF EXISTS test_move; CREATE TABLE test_move(v UInt64) ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/test_move', '{}') ORDER BY tuple() - """.format(shard_ix, replica_ix)) + """.format(shard_ix, r.name)) s0r0.query("SYSTEM STOP MERGES test_move") + s0r1.query("SYSTEM STOP MERGES test_move") s0r0.query("INSERT INTO test_move VALUES (1)") s0r0.query("INSERT INTO test_move VALUES (2)") @@ -63,14 +71,7 @@ def test_move(started_cluster): s0r0.query("SYSTEM START MERGES test_move") s0r0.query("OPTIMIZE TABLE test_move FINAL") - while True: - time.sleep(3) - - print(s0r0.query("SELECT * FROM system.part_moves_between_shards")) - - # Eventually. - if "DONE" == s0r0.query("SELECT state FROM system.part_moves_between_shards WHERE table = 'test_move'").strip(): - break + wait_for_state("DONE", s0r0, "test_move") for n in [s0r0, s0r1]: assert "1" == n.query("SELECT count() FROM test_move").strip() @@ -81,14 +82,7 @@ def test_move(started_cluster): # Move part back s1r0.query("ALTER TABLE test_move MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_0/tables/test_move'") - while True: - time.sleep(3) - - print(s1r0.query("SELECT * FROM system.part_moves_between_shards")) - - # Eventually. - if "DONE" == s1r0.query("SELECT state FROM system.part_moves_between_shards WHERE table = 'test_move'").strip(): - break + wait_for_state("DONE", s1r0, "test_move") for n in [s0r0, s0r1]: assert "2" == n.query("SELECT count() FROM test_move").strip() @@ -101,17 +95,20 @@ def test_deduplication_while_move(started_cluster): for shard_ix, rs in enumerate([[s0r0, s0r1], [s1r0, s1r1]]): for replica_ix, r in enumerate(rs): r.query(""" + DROP TABLE IF EXISTS test_deduplication; CREATE TABLE test_deduplication(v UInt64) ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/test_deduplication', '{}') ORDER BY tuple() - """.format(shard_ix, replica_ix)) + """.format(shard_ix, r.name)) r.query(""" - CREATE TABLE t_d AS test_deduplication + DROP TABLE IF EXISTS test_deduplication_d; + CREATE TABLE test_deduplication_d AS test_deduplication ENGINE Distributed('test_cluster', '', test_deduplication) """) s0r0.query("SYSTEM STOP MERGES test_deduplication") + s0r1.query("SYSTEM STOP MERGES test_deduplication") s0r0.query("INSERT INTO test_deduplication VALUES (1)") s0r0.query("INSERT INTO test_deduplication VALUES (2)") @@ -120,7 +117,8 @@ def test_deduplication_while_move(started_cluster): assert "2" == s0r0.query("SELECT count() FROM test_deduplication").strip() assert "0" == s1r0.query("SELECT count() FROM test_deduplication").strip() - s0r0.query("ALTER TABLE test_deduplication MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/test_deduplication'") + s0r0.query( + "ALTER TABLE test_deduplication MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/test_deduplication'") s0r0.query("SYSTEM START MERGES test_deduplication") expected = """ @@ -128,32 +126,363 @@ def test_deduplication_while_move(started_cluster): 2 """ - # Verify that we get consisntent result at all times while the part is moving from one shard to another. - while "DONE" != s0r0.query("SELECT state FROM system.part_moves_between_shards WHERE table = 'test_deduplication' ORDER BY create_time DESC LIMIT 1").strip(): + def deduplication_invariant_test(): n = random.choice(list(started_cluster.instances.values())) + assert TSV( + n.query("SELECT * FROM test_deduplication_d ORDER BY v", + settings={"allow_experimental_query_deduplication": 1}) + ) == TSV(expected) - assert TSV(n.query("SELECT * FROM t_d ORDER BY v", settings={ - "allow_experimental_query_deduplication": 1 - })) == TSV(expected) + deduplication_invariant = ConcurrentInvariant(deduplication_invariant_test) + deduplication_invariant.start() + + wait_for_state("DONE", s0r0, "test_deduplication") + + deduplication_invariant.stop_and_assert_no_exception() + + +def test_part_move_step_by_step(started_cluster): + for shard_ix, rs in enumerate([[s0r0, s0r1], [s1r0, s1r1]]): + for replica_ix, r in enumerate(rs): + r.query(""" + DROP TABLE IF EXISTS test_part_move_step_by_step; + CREATE TABLE test_part_move_step_by_step(v UInt64) + ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/test_part_move_step_by_step', '{}') + ORDER BY tuple() + """.format(shard_ix, r.name)) + + r.query(""" + DROP TABLE IF EXISTS test_part_move_step_by_step_d; + CREATE TABLE test_part_move_step_by_step_d AS test_part_move_step_by_step + ENGINE Distributed('test_cluster', currentDatabase(), test_part_move_step_by_step) + """) + + s0r0.query("SYSTEM STOP MERGES test_part_move_step_by_step") + s0r1.query("SYSTEM STOP MERGES test_part_move_step_by_step") + + s0r0.query("INSERT INTO test_part_move_step_by_step VALUES (1)") + s0r0.query("INSERT INTO test_part_move_step_by_step VALUES (2)") + s0r1.query("SYSTEM SYNC REPLICA test_part_move_step_by_step", timeout=20) + + assert "2" == s0r0.query("SELECT count() FROM test_part_move_step_by_step").strip() + assert "0" == s1r0.query("SELECT count() FROM test_part_move_step_by_step").strip() + + expected = """ +1 +2 +""" + + def deduplication_invariant_test(): + n = random.choice(list(started_cluster.instances.values())) + try: + assert TSV( + n.query("SELECT * FROM test_part_move_step_by_step_d ORDER BY v", + settings={"allow_experimental_query_deduplication": 1}) + ) == TSV(expected) + except QueryRuntimeException as e: + # ignore transient errors that are caused by us restarting nodes + if e.returncode not in transient_ch_errors: + raise e + + deduplication_invariant = ConcurrentInvariant(deduplication_invariant_test) + deduplication_invariant.start() + + # Stop a source replica to prevent SYNC_SOURCE succeeding. + s0r1.stop_clickhouse() + + s0r0.query( + "ALTER TABLE test_part_move_step_by_step MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/test_part_move_step_by_step'") + + # Should hang on SYNC_SOURCE until all source replicas acknowledge new pinned UUIDs. + wait_for_state("SYNC_SOURCE", s0r0, "test_part_move_step_by_step", "Some replicas haven\\'t processed event") + deduplication_invariant.assert_no_exception() + + # Start all replicas in source shard but stop a replica in destination shard + # to prevent SYNC_DESTINATION succeeding. + s1r1.stop_clickhouse() + s0r1.start_clickhouse() + + # After SYNC_SOURCE step no merges will be assigned. + s0r0.query("SYSTEM START MERGES test_part_move_step_by_step; OPTIMIZE TABLE test_part_move_step_by_step;") + s0r1.query("SYSTEM START MERGES test_part_move_step_by_step; OPTIMIZE TABLE test_part_move_step_by_step;") + + wait_for_state("SYNC_DESTINATION", s0r0, "test_part_move_step_by_step", "Some replicas haven\\'t processed event") + deduplication_invariant.assert_no_exception() + + # Start previously stopped replica in destination shard to let SYNC_DESTINATION + # succeed. + # Stop the other replica in destination shard to prevent DESTINATION_FETCH succeed. + s1r0.stop_clickhouse() + s1r1.start_clickhouse() + wait_for_state("DESTINATION_FETCH", s0r0, "test_part_move_step_by_step", "Some replicas haven\\'t processed event") + deduplication_invariant.assert_no_exception() + + # Start previously stopped replica in destination shard to let DESTINATION_FETCH + # succeed. + # Stop the other replica in destination shard to prevent DESTINATION_ATTACH succeed. + s1r1.stop_clickhouse() + s1r0.start_clickhouse() + wait_for_state("DESTINATION_ATTACH", s0r0, "test_part_move_step_by_step", "Some replicas haven\\'t processed event") + deduplication_invariant.assert_no_exception() + + # Start all replicas in destination shard to let DESTINATION_ATTACH succeed. + # Stop a source replica to prevent SOURCE_DROP succeeding. + s0r0.stop_clickhouse() + s1r1.start_clickhouse() + wait_for_state("SOURCE_DROP", s0r1, "test_part_move_step_by_step", "Some replicas haven\\'t processed event") + deduplication_invariant.assert_no_exception() + + s0r0.start_clickhouse() + wait_for_state("DONE", s0r1, "test_part_move_step_by_step") + deduplication_invariant.assert_no_exception() + + # No hung tasks in replication queue. Would timeout otherwise. + for instance in started_cluster.instances.values(): + instance.query("SYSTEM SYNC REPLICA test_part_move_step_by_step") + + assert "1" == s0r0.query("SELECT count() FROM test_part_move_step_by_step").strip() + assert "1" == s1r0.query("SELECT count() FROM test_part_move_step_by_step").strip() + + deduplication_invariant.stop_and_assert_no_exception() + + +def test_part_move_step_by_step_kill(started_cluster): + for shard_ix, rs in enumerate([[s0r0, s0r1], [s1r0, s1r1]]): + for replica_ix, r in enumerate(rs): + r.query(""" + DROP TABLE IF EXISTS test_part_move_step_by_step_kill; + CREATE TABLE test_part_move_step_by_step_kill(v UInt64) + ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/test_part_move_step_by_step_kill', '{}') + ORDER BY tuple() + """.format(shard_ix, r.name)) + + r.query(""" + DROP TABLE IF EXISTS test_part_move_step_by_step_kill_d; + CREATE TABLE test_part_move_step_by_step_kill_d AS test_part_move_step_by_step_kill + ENGINE Distributed('test_cluster', currentDatabase(), test_part_move_step_by_step_kill) + """) + + s0r0.query("SYSTEM STOP MERGES test_part_move_step_by_step_kill") + s0r1.query("SYSTEM STOP MERGES test_part_move_step_by_step_kill") + + s0r0.query("INSERT INTO test_part_move_step_by_step_kill VALUES (1)") + s0r0.query("INSERT INTO test_part_move_step_by_step_kill VALUES (2)") + s0r1.query("SYSTEM SYNC REPLICA test_part_move_step_by_step_kill", timeout=20) + + assert "2" == s0r0.query("SELECT count() FROM test_part_move_step_by_step_kill").strip() + assert "0" == s1r0.query("SELECT count() FROM test_part_move_step_by_step_kill").strip() + + expected = """ +1 +2 +""" + + def deduplication_invariant_test(): + n = random.choice(list(started_cluster.instances.values())) + try: + assert TSV( + n.query("SELECT * FROM test_part_move_step_by_step_kill_d ORDER BY v", + settings={ + "allow_experimental_query_deduplication": 1 + }) + ) == TSV(expected) + except QueryRuntimeException as e: + # ignore transient errors that are caused by us restarting nodes + if e.returncode not in transient_ch_errors: + raise e + + deduplication_invariant = ConcurrentInvariant(deduplication_invariant_test) + deduplication_invariant.start() + + # Stop a source replica to prevent SYNC_SOURCE succeeding. + s0r1.stop_clickhouse() + + s0r0.query( + "ALTER TABLE test_part_move_step_by_step_kill MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/test_part_move_step_by_step_kill'") + + # Should hang on SYNC_SOURCE until all source replicas acknowledge new pinned UUIDs. + wait_for_state("SYNC_SOURCE", s0r0, "test_part_move_step_by_step_kill", "Some replicas haven\\'t processed event") + deduplication_invariant.assert_no_exception() + + # Start all replicas in source shard but stop a replica in destination shard + # to prevent SYNC_DESTINATION succeeding. + s1r1.stop_clickhouse() + s0r1.start_clickhouse() + + # After SYNC_SOURCE step no merges will be assigned. + s0r0.query("SYSTEM START MERGES test_part_move_step_by_step_kill; OPTIMIZE TABLE test_part_move_step_by_step_kill;") + s0r1.query("SYSTEM START MERGES test_part_move_step_by_step_kill; OPTIMIZE TABLE test_part_move_step_by_step_kill;") + + wait_for_state("SYNC_DESTINATION", s0r0, "test_part_move_step_by_step_kill", + "Some replicas haven\\'t processed event") + deduplication_invariant.assert_no_exception() + + # Start previously stopped replica in destination shard to let SYNC_DESTINATION + # succeed. + # Stop the other replica in destination shard to prevent DESTINATION_FETCH succeed. + s1r0.stop_clickhouse() + s1r1.start_clickhouse() + wait_for_state("DESTINATION_FETCH", s0r0, "test_part_move_step_by_step_kill", + "Some replicas haven\\'t processed event") + + # Start previously stopped replica in destination shard to let DESTINATION_FETCH + # succeed. + # Stop the other replica in destination shard to prevent DESTINATION_ATTACH succeed. + s1r1.stop_clickhouse() + s1r0.start_clickhouse() + wait_for_state("DESTINATION_ATTACH", s0r0, "test_part_move_step_by_step_kill", + "Some replicas haven\\'t processed event") + deduplication_invariant.assert_no_exception() + + # Rollback here. + s0r0.query(""" + KILL PART_MOVE_TO_SHARD + WHERE task_uuid = (SELECT task_uuid FROM system.part_moves_between_shards WHERE table = 'test_part_move_step_by_step_kill') + """) + + wait_for_state("DESTINATION_ATTACH", s0r0, "test_part_move_step_by_step_kill", + assert_exception_msg="Some replicas haven\\'t processed event", + assert_rollback=True) + + s1r1.start_clickhouse() + + wait_for_state("CANCELLED", s0r0, "test_part_move_step_by_step_kill", assert_rollback=True) + deduplication_invariant.assert_no_exception() + + # No hung tasks in replication queue. Would timeout otherwise. + for instance in started_cluster.instances.values(): + instance.query("SYSTEM SYNC REPLICA test_part_move_step_by_step_kill") + + assert "2" == s0r0.query("SELECT count() FROM test_part_move_step_by_step_kill").strip() + assert "0" == s1r0.query("SELECT count() FROM test_part_move_step_by_step_kill").strip() + + deduplication_invariant.stop_and_assert_no_exception() def test_move_not_permitted(started_cluster): + # Verify that invariants for part compatibility are checked. + + # Tests are executed in order. Make sure cluster is up if previous test + # failed. + s0r0.start_clickhouse() + s1r0.start_clickhouse() + for ix, n in enumerate([s0r0, s1r0]): - n.query("DROP TABLE IF EXISTS not_permitted") n.query(""" - CREATE TABLE not_permitted(v_{} UInt64) - ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/not_permitted', 'r') - ORDER BY tuple() - """.format(ix, ix)) + DROP TABLE IF EXISTS not_permitted_columns; + + CREATE TABLE not_permitted_columns(v_{ix} UInt64) + ENGINE ReplicatedMergeTree('/clickhouse/shard_{ix}/tables/not_permitted_columns', 'r') + ORDER BY tuple(); + """.format(ix=ix)) - s0r0.query("INSERT INTO not_permitted VALUES (1)") + partition = "date" + if ix > 0: + partition = "v" - with pytest.raises(QueryRuntimeException) as exc: - s0r0.query("ALTER TABLE not_permitted MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/not_permitted'") + n.query(""" + DROP TABLE IF EXISTS not_permitted_partition; + CREATE TABLE not_permitted_partition(date Date, v UInt64) + ENGINE ReplicatedMergeTree('/clickhouse/shard_{ix}/tables/not_permitted_partition', 'r') + PARTITION BY ({partition}) + ORDER BY tuple(); + """.format(ix=ix, partition=partition)) - assert "DB::Exception: Table columns structure in ZooKeeper is different from local table structure." in str(exc.value) + s0r0.query("INSERT INTO not_permitted_columns VALUES (1)") + s0r0.query("INSERT INTO not_permitted_partition VALUES ('2021-09-03', 1)") - with pytest.raises(QueryRuntimeException) as exc: - s0r0.query("ALTER TABLE not_permitted MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_0/tables/not_permitted'") + with pytest.raises(QueryRuntimeException, match="DB::Exception: Source and destination are the same"): + s0r0.query("ALTER TABLE not_permitted_columns MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_0/tables/not_permitted_columns'") - assert "DB::Exception: Source and destination are the same" in str(exc.value) + with pytest.raises(QueryRuntimeException, match="DB::Exception: Table columns structure in ZooKeeper is different from local table structure."): + s0r0.query("ALTER TABLE not_permitted_columns MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/not_permitted_columns'") + + with pytest.raises(QueryRuntimeException, match="DB::Exception: Existing table metadata in ZooKeeper differs in partition key expression."): + s0r0.query("ALTER TABLE not_permitted_partition MOVE PART '20210903_0_0_0' TO SHARD '/clickhouse/shard_1/tables/not_permitted_partition'") + + +def wait_for_state(desired_state, instance, test_table, assert_exception_msg=None, assert_rollback=False): + last_debug_print_time = time.time() + + print("Waiting to reach state: {}".format(desired_state)) + if assert_exception_msg: + print(" with exception contents: {}".format(assert_exception_msg)) + if assert_rollback: + print(" and rollback: {}".format(assert_rollback)) + + while True: + tasks = TSV.toMat(instance.query( + "SELECT state, num_tries, last_exception, rollback FROM system.part_moves_between_shards WHERE table = '{}'".format( + test_table))) + assert len(tasks) == 1, "only one task expected in this test" + + if time.time() - last_debug_print_time > 30: + last_debug_print_time = time.time() + print("Current state: ", tasks) + + [state, num_tries, last_exception, rollback] = tasks[0] + + if state == desired_state: + if assert_exception_msg and int(num_tries) < 3: + # Let the task be retried a few times when expecting an exception + # to make sure the exception is persistent and the code doesn't + # accidentally continue to run when we expect it not to. + continue + + if assert_exception_msg: + assert assert_exception_msg in last_exception + + if assert_rollback: + assert int(rollback) == 1, "rollback bit isn't set" + + break + elif state in ["DONE", "CANCELLED"]: + raise Exception("Reached terminal state {}, but was waiting for {}".format(state, desired_state)) + + time.sleep(0.1) + + +class ConcurrentInvariant: + def __init__(self, invariant_test, loop_sleep=0.1): + self.invariant_test = invariant_test + self.loop_sleep = loop_sleep + + self.started = False + self.exiting = False + self.exception = None + self.thread = threading.Thread(target=self._loop) + + def start(self): + if self.started: + raise Exception('invariant thread already started') + + self.started = True + self.thread.start() + + def stop_and_assert_no_exception(self): + self._assert_started() + + self.exiting = True + self.thread.join() + + if self.exception: + raise self.exception + + def assert_no_exception(self): + self._assert_started() + + if self.exception: + raise self.exception + + def _loop(self): + try: + while not self.exiting: + self.invariant_test() + time.sleep(self.loop_sleep) + except Exception as e: + self.exiting = True + self.exception = e + + def _assert_started(self): + if not self.started: + raise Exception('invariant thread not started, forgot to call start?') From 008c3c812b09486894b1f6f617cff4de52364ce1 Mon Sep 17 00:00:00 2001 From: Kevin Michel Date: Mon, 20 Sep 2021 19:08:59 +0200 Subject: [PATCH 006/173] Allow non-replicated ALTER TABLE FETCH/ATTACH in Replicated databases `ALTER TABLE ... FETCH` and `ALTER TABLE ... ATTACH` queries were disabled in the Replicated database engine, because it could cause accidental duplication of data. This enables these queries but without replicating them. In the case of `FETCH`, the part will only be fetched on the server where the query is issued. Similarly, in the case of `ATTACH`, the attached part only needs to be available on the server where the query is issued. If the table itself is using one of the Replicated MergeTree engines, the attached data is then replicated by the table engine itself, without intervention of the database engine. This change is meant to help with live backup/restore when using the Replicated database engine, using FREEZE for backup and ATTACH for restore. --- .../en/engines/database-engines/replicated.md | 2 + src/Interpreters/InterpreterAlterQuery.cpp | 4 +- src/Parsers/ASTAlterQuery.cpp | 11 +++++ src/Parsers/ASTAlterQuery.h | 4 ++ .../test_replicated_database/test.py | 49 ++++++++++++++++++- 5 files changed, 68 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/database-engines/replicated.md b/docs/en/engines/database-engines/replicated.md index 5ce73604207..67671e11cab 100644 --- a/docs/en/engines/database-engines/replicated.md +++ b/docs/en/engines/database-engines/replicated.md @@ -35,6 +35,8 @@ The [system.clusters](../../operations/system-tables/clusters.md) system table c When creating a new replica of the database, this replica creates tables by itself. If the replica has been unavailable for a long time and has lagged behind the replication log — it checks its local metadata with the current metadata in ZooKeeper, moves the extra tables with data to a separate non-replicated database (so as not to accidentally delete anything superfluous), creates the missing tables, updates the table names if they have been renamed. The data is replicated at the `ReplicatedMergeTree` level, i.e. if the table is not replicated, the data will not be replicated (the database is responsible only for metadata). +[`ALTER TABLE FETCH`](../../sql-reference/statements/alter/partition.md) and [`ALTER TABLE ATTACH`](../../sql-reference/statements/alter/partition.md) queries are allowed but not replicated. The database engine will only add the partition/part to the current replica. However, if the table itself uses a Replicated table engine, then the data will be replicated after using `ATTACH`. + ## Usage Example {#usage-example} Creating a cluster with three hosts: diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 08b9846b325..e2b2d487be5 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -55,7 +55,9 @@ BlockIO InterpreterAlterQuery::execute() DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name); if (typeid_cast(database.get()) - && !getContext()->getClientInfo().is_replicated_database_internal) + && !getContext()->getClientInfo().is_replicated_database_internal + && !alter.isAttachAlter() + && !alter.isFetchAlter()) { auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name); guard->releaseTableLock(); diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 4b70f25f5c5..140ef972eeb 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -450,6 +450,17 @@ bool ASTAlterQuery::isFreezeAlter() const || isOneCommandTypeOnly(ASTAlterCommand::UNFREEZE_PARTITION) || isOneCommandTypeOnly(ASTAlterCommand::UNFREEZE_ALL); } +bool ASTAlterQuery::isAttachAlter() const +{ + return isOneCommandTypeOnly(ASTAlterCommand::ATTACH_PARTITION); +} + +bool ASTAlterQuery::isFetchAlter() const +{ + return isOneCommandTypeOnly(ASTAlterCommand::FETCH_PARTITION); +} + + /** Get the text that identifies this element. */ String ASTAlterQuery::getID(char delim) const { diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index a5410e797ce..432f0c887fd 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -218,6 +218,10 @@ public: bool isFreezeAlter() const; + bool isAttachAlter() const; + + bool isFetchAlter() const; + String getID(char) const override; ASTPtr clone() const override; diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 450caafb9ba..ff400f4466b 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1,3 +1,5 @@ +import os +import shutil import time import re import pytest @@ -16,7 +18,7 @@ snapshot_recovering_node = cluster.add_instance('snapshot_recovering_node', main all_nodes = [main_node, dummy_node, competing_node, snapshotting_node, snapshot_recovering_node] -uuid_regex = re.compile("[0-9a-f]{8}\-[0-9a-f]{4}\-[0-9a-f]{4}\-[0-9a-f]{4}\-[0-9a-f]{12}") +uuid_regex = re.compile("[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}") def assert_create_query(nodes, table_name, expected): replace_uuid = lambda x: re.sub(uuid_regex, "uuid", x) query = "show create table {}".format(table_name) @@ -100,6 +102,51 @@ def test_simple_alter_table(started_cluster, engine): assert_create_query([main_node, dummy_node, competing_node], name, expected) +def get_table_uuid(database, name): + return main_node.query(f"SELECT uuid FROM system.tables WHERE database = '{database}' and name = '{name}'").strip() + + +@pytest.fixture(scope="module", name="attachable_part") +def fixture_attachable_part(started_cluster): + main_node.query(f"CREATE DATABASE testdb_attach_atomic ENGINE = Atomic") + main_node.query(f"CREATE TABLE testdb_attach_atomic.test (CounterID UInt32) ENGINE = MergeTree ORDER BY (CounterID)") + main_node.query(f"INSERT INTO testdb_attach_atomic.test VALUES (123)") + main_node.query(f"ALTER TABLE testdb_attach_atomic.test FREEZE WITH NAME 'test_attach'") + table_uuid = get_table_uuid("testdb_attach_atomic", "test") + return os.path.join(main_node.path, f"database/shadow/test_attach/store/{table_uuid[:3]}/{table_uuid}/all_1_1_0") + + + +@pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) +def test_alter_attach(started_cluster, attachable_part, engine): + name = "alter_attach_test_{}".format(engine) + main_node.query(f"CREATE TABLE testdb.{name} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)") + table_uuid = get_table_uuid("testdb", name) + # Provide and attach a part to the main node + shutil.copytree( + attachable_part, os.path.join(main_node.path, f"database/store/{table_uuid[:3]}/{table_uuid}/detached/all_1_1_0") + ) + main_node.query(f"ALTER TABLE testdb.{name} ATTACH PART 'all_1_1_0'") + # On the main node, data is attached + assert main_node.query(f"SELECT CounterID FROM testdb.{name}") == "123\n" + # On the other node, data is replicated only if using a Replicated table engine + if engine == "ReplicatedMergeTree": + assert dummy_node.query(f"SELECT CounterID FROM testdb.{name}") == "123\n" + else: + assert dummy_node.query(f"SELECT CounterID FROM testdb.{name}") == "" + + +def test_alter_fetch(started_cluster): + main_node.query("CREATE TABLE testdb.fetch_source (CounterID UInt32) ENGINE = ReplicatedMergeTree ORDER BY (CounterID)") + main_node.query("CREATE TABLE testdb.fetch_target (CounterID UInt32) ENGINE = ReplicatedMergeTree ORDER BY (CounterID)") + main_node.query("INSERT INTO testdb.fetch_source VALUES (123)") + table_uuid = get_table_uuid("testdb", "fetch_source") + main_node.query(f"ALTER TABLE testdb.fetch_target FETCH PART 'all_0_0_0' FROM '/clickhouse/tables/{table_uuid}/{{shard}}' ") + detached_parts_query = "SELECT name FROM system.detached_parts WHERE database='testdb' AND table='fetch_target'" + assert main_node.query(detached_parts_query) == "all_0_0_0\n" + assert dummy_node.query(detached_parts_query) == "" + + def test_alters_from_different_replicas(started_cluster): # test_alters_from_different_replicas competing_node.query("CREATE DATABASE IF NOT EXISTS testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica3');") From aa3f4003c82b80331468e789e31dfa7cd7ad0420 Mon Sep 17 00:00:00 2001 From: Kevin Michel Date: Fri, 24 Sep 2021 12:51:52 +0200 Subject: [PATCH 007/173] Fix hanging DDL queries on Replicated database There was a race condition when issuing a DDL query on a replica just after a new replica was added. If the DDL query is issued after the new replica adds itself to the list of replicas, but before the new replica has finished its recovery, then the first replica adds the new replica to the list of replicas to wait to confirm the query was replicated. Meanwhile, the new replica is still in recovery and applies queries from the /metadata snapshot. When it's done, it bumps its log_ptr without marking the corresponding log entries (if any) as finished. The first replica then waits until distributed_ddl_task_timeout expires and wrongly assumes the query was not replicated. The issue is fixed by remembering the max_log_ptr at the exact point where the replica adds itself to the list of replicas, then mark as finished all queries that happened between that max_log_ptr and the max_log_ptr of the metadata snapshot used in recovery. The bug was randomly observed during a downstream test. It can be reproduced more easily by inserting a sleep of a few seconds at the end of createReplicaNodesInZooKeeper, enough to have time to issue a DDL query on the first replica. --- src/Databases/DatabaseReplicated.cpp | 43 +++++++++++++++++++++++++--- src/Databases/DatabaseReplicated.h | 1 + 2 files changed, 40 insertions(+), 4 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index da03eb6aba6..42389941c98 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -298,10 +298,30 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt /// Write host name to replica_path, it will protect from multiple replicas with the same name auto host_id = getHostID(getContext(), db_uuid); - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCreateRequest(replica_path, host_id, zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_ptr", "0", zkutil::CreateMode::Persistent)); - current_zookeeper->multi(ops); + for (int attempts = 10; attempts > 0; --attempts) + { + Coordination::Stat stat; + String max_log_ptr_str = current_zookeeper->get(zookeeper_path + "/max_log_ptr", &stat); + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest(replica_path, host_id, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_ptr", "0", zkutil::CreateMode::Persistent)); + /// In addition to creating the replica nodes, we record the max_log_ptr at the instant where + /// we declared ourself as an existing replica. We'll need this during recoverLostReplica to + /// notify other nodes that issued new queries while this node was recovering. + ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/max_log_ptr", stat.version)); + Coordination::Responses responses; + const auto code = current_zookeeper->tryMulti(ops, responses); + if (code == Coordination::Error::ZOK) + { + max_log_ptr_at_creation = parse(max_log_ptr_str); + break; + } + else if (code == Coordination::Error::ZNODEEXISTS || attempts == 1) + { + /// If its our last attempt, or if the replica already exists, fail immediately. + zkutil::KeeperMultiException::check(code, ops, responses); + } + } createEmptyLogEntry(current_zookeeper); } @@ -613,6 +633,21 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep InterpreterCreateQuery(query_ast, create_query_context).execute(); } + if (max_log_ptr_at_creation != 0) + { + /// If the replica is new and some of the queries applied during recovery + /// where issued after the replica was created, then other nodes might be + /// waiting for this node to notify them that the query was applied. + for (UInt32 ptr = max_log_ptr_at_creation; ptr <= max_log_ptr; ++ptr) + { + auto entry_name = DDLTaskBase::getLogEntryName(ptr); + auto path = fs::path(zookeeper_path) / "log" / entry_name / "finished" / getFullReplicaName(); + auto status = ExecutionStatus(0).serializeText(); + auto res = current_zookeeper->tryCreate(path, status, zkutil::CreateMode::Persistent); + if (res == Coordination::Error::ZOK) + LOG_INFO(log, "Marked recovered {} as finished", entry_name); + } + } current_zookeeper->set(replica_path + "/log_ptr", toString(max_log_ptr)); } diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 1e0daeed07e..849c33dfbcc 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -90,6 +90,7 @@ private: std::atomic_bool is_readonly = true; std::unique_ptr ddl_worker; + UInt32 max_log_ptr_at_creation = 0; mutable ClusterPtr cluster; }; From 14b1c438ccf2dff7fedd4c9b8b1f64d5ba65da04 Mon Sep 17 00:00:00 2001 From: Kevin Michel Date: Tue, 21 Sep 2021 17:47:48 +0200 Subject: [PATCH 008/173] Enable all tests using ATTACH/FETCH on Replicated database --- .../queries/0_stateless/00626_replace_partition_from_table.sql | 3 +-- .../00626_replace_partition_from_table_zookeeper.sh | 3 +-- tests/queries/0_stateless/00753_alter_attach.sql | 3 +-- tests/queries/0_stateless/00955_test_final_mark.sql | 3 +-- tests/queries/0_stateless/00955_test_final_mark_use.sh | 3 +-- tests/queries/0_stateless/01015_attach_part.sql | 3 +-- tests/queries/0_stateless/01021_only_tuple_columns.sql | 3 +-- .../queries/0_stateless/01060_shutdown_table_after_detach.sql | 3 +-- tests/queries/0_stateless/01130_in_memory_parts_partitons.sql | 3 +-- .../0_stateless/01417_freeze_partition_verbose_zookeeper.sh | 2 +- tests/queries/0_stateless/01451_detach_drop_part.sql | 3 --- .../0_stateless/01451_replicated_detach_drop_part_long.sql | 2 +- .../01650_fetch_patition_with_macro_in_zk_path_long.sql | 3 +-- tests/queries/0_stateless/02009_array_join_partition.sql | 3 --- .../0_stateless/02012_changed_enum_type_non_replicated.sql | 3 +-- .../queries/0_stateless/02012_zookeeper_changed_enum_type.sql | 2 +- .../02012_zookeeper_changed_enum_type_incompatible.sql | 2 +- tests/queries/1_stateful/00054_merge_tree_partitions.sql | 3 --- .../queries/1_stateful/00152_insert_different_granularity.sql | 2 +- 19 files changed, 16 insertions(+), 36 deletions(-) diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table.sql b/tests/queries/0_stateless/00626_replace_partition_from_table.sql index 1e3e07f8274..958d2184c46 100644 --- a/tests/queries/0_stateless/00626_replace_partition_from_table.sql +++ b/tests/queries/0_stateless/00626_replace_partition_from_table.sql @@ -1,5 +1,4 @@ --- Tags: no-replicated-database, no-parallel --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: no-parallel DROP TABLE IF EXISTS src; DROP TABLE IF EXISTS dst; diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index 7b3f787eeab..d78f93d6bb3 100755 --- a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -1,6 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-replicated-database, no-parallel -# Tag no-replicated-database: Unsupported type of ALTER query +# Tags: zookeeper, no-parallel # Because REPLACE PARTITION does not forces immediate removal of replaced data parts from local filesystem # (it tries to do it as quick as possible, but it still performed in separate thread asynchronously) diff --git a/tests/queries/0_stateless/00753_alter_attach.sql b/tests/queries/0_stateless/00753_alter_attach.sql index a9661d3a304..ca43fb3aeae 100644 --- a/tests/queries/0_stateless/00753_alter_attach.sql +++ b/tests/queries/0_stateless/00753_alter_attach.sql @@ -1,5 +1,4 @@ --- Tags: no-replicated-database, no-parallel --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: no-parallel DROP TABLE IF EXISTS alter_attach; CREATE TABLE alter_attach (x UInt64, p UInt8) ENGINE = MergeTree ORDER BY tuple() PARTITION BY p; diff --git a/tests/queries/0_stateless/00955_test_final_mark.sql b/tests/queries/0_stateless/00955_test_final_mark.sql index 3c92d96e9a9..6615c945ef8 100644 --- a/tests/queries/0_stateless/00955_test_final_mark.sql +++ b/tests/queries/0_stateless/00955_test_final_mark.sql @@ -1,5 +1,4 @@ --- Tags: no-replicated-database, no-parallel --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: no-parallel SET send_logs_level = 'fatal'; diff --git a/tests/queries/0_stateless/00955_test_final_mark_use.sh b/tests/queries/0_stateless/00955_test_final_mark_use.sh index 8642011b670..2c3219fbffd 100755 --- a/tests/queries/0_stateless/00955_test_final_mark_use.sh +++ b/tests/queries/0_stateless/00955_test_final_mark_use.sh @@ -1,6 +1,5 @@ #!/usr/bin/env bash -# Tags: no-replicated-database, no-parallel -# Tag no-replicated-database: Unsupported type of ALTER query +# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01015_attach_part.sql b/tests/queries/0_stateless/01015_attach_part.sql index 3d2058f757e..6b786bfbab9 100644 --- a/tests/queries/0_stateless/01015_attach_part.sql +++ b/tests/queries/0_stateless/01015_attach_part.sql @@ -1,5 +1,4 @@ --- Tags: no-replicated-database, no-parallel --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: no-parallel DROP TABLE IF EXISTS table_01; diff --git a/tests/queries/0_stateless/01021_only_tuple_columns.sql b/tests/queries/0_stateless/01021_only_tuple_columns.sql index a0be59c3120..02db21bc0b2 100644 --- a/tests/queries/0_stateless/01021_only_tuple_columns.sql +++ b/tests/queries/0_stateless/01021_only_tuple_columns.sql @@ -1,5 +1,4 @@ --- Tags: no-replicated-database, no-parallel --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: no-parallel CREATE TABLE test ( diff --git a/tests/queries/0_stateless/01060_shutdown_table_after_detach.sql b/tests/queries/0_stateless/01060_shutdown_table_after_detach.sql index 5e0ed8b298d..bfe928d7003 100644 --- a/tests/queries/0_stateless/01060_shutdown_table_after_detach.sql +++ b/tests/queries/0_stateless/01060_shutdown_table_after_detach.sql @@ -1,5 +1,4 @@ --- Tags: no-replicated-database, no-parallel --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: no-parallel DROP TABLE IF EXISTS test; CREATE TABLE test Engine = MergeTree ORDER BY number AS SELECT number, toString(rand()) x from numbers(10000000); diff --git a/tests/queries/0_stateless/01130_in_memory_parts_partitons.sql b/tests/queries/0_stateless/01130_in_memory_parts_partitons.sql index a0e8d24eff6..aa6f281e0eb 100644 --- a/tests/queries/0_stateless/01130_in_memory_parts_partitons.sql +++ b/tests/queries/0_stateless/01130_in_memory_parts_partitons.sql @@ -1,5 +1,4 @@ --- Tags: no-replicated-database, no-parallel --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: no-parallel DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh b/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh index 5b936b3f2ba..01678d1b500 100755 --- a/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh +++ b/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh @@ -1,6 +1,6 @@ #!/usr/bin/env bash # Tags: zookeeper, no-replicated-database, no-parallel -# Tag no-replicated-database: Unsupported type of ALTER query +# Tag no-replicated-database: Fails due to additional replicas or shards CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01451_detach_drop_part.sql b/tests/queries/0_stateless/01451_detach_drop_part.sql index 0bc7e13d243..a285730e45f 100644 --- a/tests/queries/0_stateless/01451_detach_drop_part.sql +++ b/tests/queries/0_stateless/01451_detach_drop_part.sql @@ -1,6 +1,3 @@ --- Tags: no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query - DROP TABLE IF EXISTS mt_01451; CREATE TABLE mt_01451 (v UInt8) ENGINE = MergeTree() order by tuple(); diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql index eb4d5e43115..9f9d1db78f1 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql @@ -1,5 +1,5 @@ -- Tags: long, replica, no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tag no-replicated-database: Fails due to additional replicas or shards SET replication_alter_partitions_sync = 2; diff --git a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql index f9b9181b5b7..ce48ad6a02a 100644 --- a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql +++ b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql @@ -1,5 +1,4 @@ --- Tags: long, no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: long DROP TABLE IF EXISTS test_01640; DROP TABLE IF EXISTS restore_01640; diff --git a/tests/queries/0_stateless/02009_array_join_partition.sql b/tests/queries/0_stateless/02009_array_join_partition.sql index 1b99deec0f9..a78efe96f66 100644 --- a/tests/queries/0_stateless/02009_array_join_partition.sql +++ b/tests/queries/0_stateless/02009_array_join_partition.sql @@ -1,6 +1,3 @@ --- Tags: no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query - CREATE TABLE table_2009_part (`i` Int64, `d` Date, `s` String) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY i; ALTER TABLE table_2009_part ATTACH PARTITION tuple(arrayJoin([0, 1])); -- {serverError 248} diff --git a/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.sql b/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.sql index 5f53984a8e4..50d5bfe0d39 100644 --- a/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.sql +++ b/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.sql @@ -1,5 +1,4 @@ --- Tags: replica, no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: replica create table enum_alter_issue (a Enum8('one' = 1, 'two' = 2)) engine = MergeTree() ORDER BY a; insert into enum_alter_issue values ('one'), ('two'); diff --git a/tests/queries/0_stateless/02012_zookeeper_changed_enum_type.sql b/tests/queries/0_stateless/02012_zookeeper_changed_enum_type.sql index 790db4c5c3d..5dc9ef8ca17 100644 --- a/tests/queries/0_stateless/02012_zookeeper_changed_enum_type.sql +++ b/tests/queries/0_stateless/02012_zookeeper_changed_enum_type.sql @@ -1,5 +1,5 @@ -- Tags: zookeeper, no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tag no-replicated-database: Fails due to additional replicas or shards create table enum_alter_issue (a Enum8('one' = 1, 'two' = 2), b Int) engine = ReplicatedMergeTree('/clickhouse/tables/{database}/test_02012/enum_alter_issue', 'r1') diff --git a/tests/queries/0_stateless/02012_zookeeper_changed_enum_type_incompatible.sql b/tests/queries/0_stateless/02012_zookeeper_changed_enum_type_incompatible.sql index 697758be0c8..e236e6d2767 100644 --- a/tests/queries/0_stateless/02012_zookeeper_changed_enum_type_incompatible.sql +++ b/tests/queries/0_stateless/02012_zookeeper_changed_enum_type_incompatible.sql @@ -1,5 +1,5 @@ -- Tags: zookeeper, no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tag no-replicated-database: Fails due to additional replicas or shards drop table if exists enum_alter_issue; create table enum_alter_issue (a Enum16('one' = 1, 'two' = 2), b Int) diff --git a/tests/queries/1_stateful/00054_merge_tree_partitions.sql b/tests/queries/1_stateful/00054_merge_tree_partitions.sql index 91e67ac857a..73020952977 100644 --- a/tests/queries/1_stateful/00054_merge_tree_partitions.sql +++ b/tests/queries/1_stateful/00054_merge_tree_partitions.sql @@ -1,6 +1,3 @@ --- Tags: no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query - DROP TABLE IF EXISTS test.partitions; CREATE TABLE test.partitions (EventDate Date, CounterID UInt32) ENGINE = MergeTree(EventDate, CounterID, 8192); INSERT INTO test.partitions SELECT EventDate + UserID % 365 AS EventDate, CounterID FROM test.hits WHERE CounterID = 1704509; diff --git a/tests/queries/1_stateful/00152_insert_different_granularity.sql b/tests/queries/1_stateful/00152_insert_different_granularity.sql index 5c802642a5c..da99173ed96 100644 --- a/tests/queries/1_stateful/00152_insert_different_granularity.sql +++ b/tests/queries/1_stateful/00152_insert_different_granularity.sql @@ -1,5 +1,5 @@ -- Tags: no-tsan, no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tag no-replicated-database: Fails due to additional replicas or shards DROP TABLE IF EXISTS fixed_granularity_table; From 045d4f2d3278653b296397e78a49c238978024a9 Mon Sep 17 00:00:00 2001 From: Kevin Michel Date: Sun, 26 Sep 2021 18:47:24 +0200 Subject: [PATCH 009/173] Don't replicate DROP/DETACH/DROP DETACHED on Replicated database --- .../en/engines/database-engines/replicated.md | 2 +- src/Interpreters/InterpreterAlterQuery.cpp | 3 +- src/Parsers/ASTAlterQuery.cpp | 5 ++ src/Parsers/ASTAlterQuery.h | 2 + .../configs/settings.xml | 1 + .../test_replicated_database/test.py | 51 +++++++++++++++++++ 6 files changed, 62 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/database-engines/replicated.md b/docs/en/engines/database-engines/replicated.md index 67671e11cab..bdc17d32393 100644 --- a/docs/en/engines/database-engines/replicated.md +++ b/docs/en/engines/database-engines/replicated.md @@ -35,7 +35,7 @@ The [system.clusters](../../operations/system-tables/clusters.md) system table c When creating a new replica of the database, this replica creates tables by itself. If the replica has been unavailable for a long time and has lagged behind the replication log — it checks its local metadata with the current metadata in ZooKeeper, moves the extra tables with data to a separate non-replicated database (so as not to accidentally delete anything superfluous), creates the missing tables, updates the table names if they have been renamed. The data is replicated at the `ReplicatedMergeTree` level, i.e. if the table is not replicated, the data will not be replicated (the database is responsible only for metadata). -[`ALTER TABLE FETCH`](../../sql-reference/statements/alter/partition.md) and [`ALTER TABLE ATTACH`](../../sql-reference/statements/alter/partition.md) queries are allowed but not replicated. The database engine will only add the partition/part to the current replica. However, if the table itself uses a Replicated table engine, then the data will be replicated after using `ATTACH`. +[`ALTER TABLE ATTACH|FETCH|DROP|DROP DETACHED|DETACH PARTITION|PART`](../../sql-reference/statements/alter/partition.md) queries are allowed but not replicated. The database engine will only add/fetch/remove the partition/part to the current replica. However, if the table itself uses a Replicated table engine, then the data will be replicated after using `ATTACH`. ## Usage Example {#usage-example} diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index e2b2d487be5..a1d083e298d 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -57,7 +57,8 @@ BlockIO InterpreterAlterQuery::execute() if (typeid_cast(database.get()) && !getContext()->getClientInfo().is_replicated_database_internal && !alter.isAttachAlter() - && !alter.isFetchAlter()) + && !alter.isFetchAlter() + && !alter.isDropPartitionAlter()) { auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name); guard->releaseTableLock(); diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 140ef972eeb..f3f83258ac8 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -460,6 +460,11 @@ bool ASTAlterQuery::isFetchAlter() const return isOneCommandTypeOnly(ASTAlterCommand::FETCH_PARTITION); } +bool ASTAlterQuery::isDropPartitionAlter() const +{ + return isOneCommandTypeOnly(ASTAlterCommand::DROP_PARTITION) || isOneCommandTypeOnly(ASTAlterCommand::DROP_DETACHED_PARTITION); +} + /** Get the text that identifies this element. */ String ASTAlterQuery::getID(char delim) const diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 432f0c887fd..2f53026c8d8 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -222,6 +222,8 @@ public: bool isFetchAlter() const; + bool isDropPartitionAlter() const; + String getID(char) const override; ASTPtr clone() const override; diff --git a/tests/integration/test_replicated_database/configs/settings.xml b/tests/integration/test_replicated_database/configs/settings.xml index 7f45502e20d..83e83a47c0b 100644 --- a/tests/integration/test_replicated_database/configs/settings.xml +++ b/tests/integration/test_replicated_database/configs/settings.xml @@ -1,6 +1,7 @@ + 1 1 1 diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index ff400f4466b..fa1e9cf49fa 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -136,6 +136,57 @@ def test_alter_attach(started_cluster, attachable_part, engine): assert dummy_node.query(f"SELECT CounterID FROM testdb.{name}") == "" +@pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) +def test_alter_drop_part(started_cluster, engine): + table = f"alter_drop_{engine}" + part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" + main_node.query(f"CREATE TABLE testdb.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)") + main_node.query(f"INSERT INTO testdb.{table} VALUES (123)") + if engine == "MergeTree": + dummy_node.query(f"INSERT INTO testdb.{table} VALUES (456)") + main_node.query(f"ALTER TABLE testdb.{table} DROP PART '{part_name}'") + assert main_node.query(f"SELECT CounterID FROM testdb.{table}") == "" + if engine == "ReplicatedMergeTree": + # The DROP operation is still replicated at the table engine level + assert dummy_node.query(f"SELECT CounterID FROM testdb.{table}") == "" + else: + assert dummy_node.query(f"SELECT CounterID FROM testdb.{table}") == "456\n" + + +@pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) +def test_alter_detach_part(started_cluster, engine): + table = f"alter_detach_{engine}" + part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" + main_node.query(f"CREATE TABLE testdb.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)") + main_node.query(f"INSERT INTO testdb.{table} VALUES (123)") + if engine == "MergeTree": + dummy_node.query(f"INSERT INTO testdb.{table} VALUES (456)") + main_node.query(f"ALTER TABLE testdb.{table} DETACH PART '{part_name}'") + detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='testdb' AND table='{table}'" + assert main_node.query(detached_parts_query) == f"{part_name}\n" + if engine == "ReplicatedMergeTree": + # The detach operation is still replicated at the table engine level + assert dummy_node.query(detached_parts_query) == f"{part_name}\n" + else: + assert dummy_node.query(detached_parts_query) == "" + + +@pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) +def test_alter_drop_detached_part(started_cluster, engine): + table = f"alter_drop_detached_{engine}" + part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" + main_node.query(f"CREATE TABLE testdb.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)") + main_node.query(f"INSERT INTO testdb.{table} VALUES (123)") + main_node.query(f"ALTER TABLE testdb.{table} DETACH PART '{part_name}'") + if engine == "MergeTree": + dummy_node.query(f"INSERT INTO testdb.{table} VALUES (456)") + dummy_node.query(f"ALTER TABLE testdb.{table} DETACH PART '{part_name}'") + main_node.query(f"ALTER TABLE testdb.{table} DROP DETACHED PART '{part_name}'") + detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='testdb' AND table='{table}'" + assert main_node.query(detached_parts_query) == "" + assert dummy_node.query(detached_parts_query) == f"{part_name}\n" + + def test_alter_fetch(started_cluster): main_node.query("CREATE TABLE testdb.fetch_source (CounterID UInt32) ENGINE = ReplicatedMergeTree ORDER BY (CounterID)") main_node.query("CREATE TABLE testdb.fetch_target (CounterID UInt32) ENGINE = ReplicatedMergeTree ORDER BY (CounterID)") From 72473e26e2bda8ef019bfdfe95b68e80ba6c9e86 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Wed, 29 Sep 2021 21:08:20 +0300 Subject: [PATCH 010/173] update example --- docs/en/operations/system-tables/replicas.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/replicas.md b/docs/en/operations/system-tables/replicas.md index 251c7e65537..7bca3e78ff9 100644 --- a/docs/en/operations/system-tables/replicas.md +++ b/docs/en/operations/system-tables/replicas.md @@ -45,6 +45,9 @@ last_queue_update: 2020-02-20 08:34:32 absolute_delay: 0 total_replicas: 2 active_replicas: 2 +last_queue_update_exception: +zookeeper_exception: +replica_is_active: {'example01-06-1.yandex.ru':1} ``` Columns: @@ -82,8 +85,9 @@ The next 4 columns have a non-zero value only where there is an active session w - `absolute_delay` (`UInt64`) - How big lag in seconds the current replica has. - `total_replicas` (`UInt8`) - The total number of known replicas of this table. - `active_replicas` (`UInt8`) - The number of replicas of this table that have a session in ZooKeeper (i.e., the number of functioning replicas). -- `replica_is_active` ([Map(String, UInt8)](../../sql-reference/data-types/map.md)) — Map between replica name and is replica active. - `last_queue_update_exception` (`String`) - When the queue contains broken entries. Especially important when ClickHouse breaks backward compatibility between versions and log entries written by newer versions aren't parseable by old versions. +- `zookeeper_exception` (`String`) - The last exception message, got if the error happened when fetching the info from ZooKeeper. +- `replica_is_active` ([Map(String, UInt8)](../../sql-reference/data-types/map.md)) — Map between replica name and is replica active. If you request all the columns, the table may work a bit slowly, since several reads from ZooKeeper are made for each row. If you do not request the last 4 columns (log_max_index, log_pointer, total_replicas, active_replicas), the table works quickly. From b336947f77feb0fabc82ddc1522d9639f8ee3947 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Fri, 1 Oct 2021 16:06:10 +0300 Subject: [PATCH 011/173] new example --- docs/en/operations/system-tables/replicas.md | 68 ++++++++++---------- 1 file changed, 35 insertions(+), 33 deletions(-) diff --git a/docs/en/operations/system-tables/replicas.md b/docs/en/operations/system-tables/replicas.md index 7bca3e78ff9..c959100b6e4 100644 --- a/docs/en/operations/system-tables/replicas.md +++ b/docs/en/operations/system-tables/replicas.md @@ -13,41 +13,43 @@ FORMAT Vertical ``` ``` text +Query id: dc6dcbcb-dc28-4df9-ae27-4354f5b3b13e + Row 1: ────── -database: merge -table: visits -engine: ReplicatedCollapsingMergeTree -is_leader: 1 -can_become_leader: 1 -is_readonly: 0 -is_session_expired: 0 -future_parts: 1 -parts_to_check: 0 -zookeeper_path: /clickhouse/tables/01-06/visits -replica_name: example01-06-1.yandex.ru -replica_path: /clickhouse/tables/01-06/visits/replicas/example01-06-1.yandex.ru -columns_version: 9 -queue_size: 1 -inserts_in_queue: 0 -merges_in_queue: 1 -part_mutations_in_queue: 0 -queue_oldest_time: 2020-02-20 08:34:30 -inserts_oldest_time: 1970-01-01 00:00:00 -merges_oldest_time: 2020-02-20 08:34:30 -part_mutations_oldest_time: 1970-01-01 00:00:00 -oldest_part_to_get: -oldest_part_to_merge_to: 20200220_20284_20840_7 -oldest_part_to_mutate_to: -log_max_index: 596273 -log_pointer: 596274 -last_queue_update: 2020-02-20 08:34:32 -absolute_delay: 0 -total_replicas: 2 -active_replicas: 2 -last_queue_update_exception: -zookeeper_exception: -replica_is_active: {'example01-06-1.yandex.ru':1} +database: default +table: test +engine: ReplicatedMergeTree +is_leader: 1 +can_become_leader: 1 +is_readonly: 0 +is_session_expired: 0 +future_parts: 0 +parts_to_check: 0 +zookeeper_path: /test/t +replica_name: 1 +replica_path: /test/t/replicas/1 +columns_version: -1 +queue_size: 0 +inserts_in_queue: 0 +merges_in_queue: 0 +part_mutations_in_queue: 0 +queue_oldest_time: 1970-01-01 03:00:00 +inserts_oldest_time: 1970-01-01 03:00:00 +merges_oldest_time: 1970-01-01 03:00:00 +part_mutations_oldest_time: 1970-01-01 03:00:00 +oldest_part_to_get: +oldest_part_to_merge_to: +oldest_part_to_mutate_to: +log_max_index: 0 +log_pointer: 0 +last_queue_update: 1970-01-01 03:00:00 +absolute_delay: 0 +total_replicas: 1 +active_replicas: 1 +last_queue_update_exception: +zookeeper_exception: +replica_is_active: {'1':1} ``` Columns: From 6169b180cb9374acc1ab87e57e7975724cbfaf8c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 4 Oct 2021 17:27:44 +0300 Subject: [PATCH 012/173] Fix backward compatibility after #21196. --- src/Core/Defines.h | 4 +- src/Interpreters/ClientInfo.h | 8 +++ src/Interpreters/InterpreterSelectQuery.cpp | 14 +++++ src/Server/TCPHandler.cpp | 5 ++ .../test_aggregate_fixed_key.py | 61 +++++++++++++++++++ 5 files changed, 91 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py diff --git a/src/Core/Defines.h b/src/Core/Defines.h index ce5c9098399..3620c1dfcfe 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -63,7 +63,9 @@ /// Minimum revision with exactly the same set of aggregation methods and rules to select them. /// Two-level (bucketed) aggregation is incompatible if servers are inconsistent in these rules /// (keys will be placed in different buckets and result will not be fully aggregated). -#define DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 54431 +#define DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 54456 +#define DBMS_MIN_MAJOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 21 +#define DBMS_MIN_MINOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 4 #define DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA 54410 #define DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE 54405 diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index 71570778645..716c66b8b71 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -83,6 +83,14 @@ public: UInt64 client_version_patch = 0; unsigned client_tcp_protocol_version = 0; + /// In case of distributed query, client info for query is actually a client info of client. + /// In order to get a version of server-initiator, use connection_ values. + /// Also for tcp only. + UInt64 connection_client_version_major = 0; + UInt64 connection_client_version_minor = 0; + UInt64 connection_client_version_patch = 0; + unsigned connection_tcp_protocol_version = 0; + /// For http HTTPMethod http_method = HTTPMethod::UNKNOWN; String http_user_agent; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 563e9ba8997..9ef35a774bf 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -75,6 +75,7 @@ #include #include +#include #include #include #include @@ -2609,6 +2610,19 @@ void InterpreterSelectQuery::initSettings() auto & query = getSelectQuery(); if (query.settings()) InterpreterSetQuery(query.settings(), context).executeForCurrentContext(); + + auto & client_info = context->getClientInfo(); + auto min_major = DBMS_MIN_MAJOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD; + auto min_minor = DBMS_MIN_MINOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD; + + if (client_info.query_kind == ClientInfo::QueryKind::SECONDARY_QUERY && + std::forward_as_tuple(client_info.connection_client_version_major, client_info.connection_client_version_minor) < std::forward_as_tuple(min_major, min_minor)) + { + /// Disable two-level aggregation due to version incompatibility. + context->setSetting("group_by_two_level_threshold", Field(0)); + context->setSetting("group_by_two_level_threshold_bytes", Field(0)); + + } } } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 5415bf96443..a139ab5c100 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -944,6 +944,11 @@ void TCPHandler::receiveHello() client_info.client_version_patch = client_version_patch; client_info.client_tcp_protocol_version = client_tcp_protocol_version; + client_info.connection_client_version_major = client_version_major; + client_info.connection_client_version_minor = client_version_minor; + client_info.connection_client_version_patch = client_version_patch; + client_info.connection_tcp_protocol_version = client_tcp_protocol_version; + is_interserver_mode = (user == USER_INTERSERVER_MARKER); if (is_interserver_mode) { diff --git a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py new file mode 100644 index 00000000000..8819be527fd --- /dev/null +++ b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py @@ -0,0 +1,61 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', with_zookeeper=True, image='yandex/clickhouse-server', tag='21.3', with_installed_binary=True) +node2 = cluster.add_instance('node2', with_zookeeper=True, image='yandex/clickhouse-server') +node3 = cluster.add_instance('node3', with_zookeeper=True, image='yandex/clickhouse-server') + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_two_level_merge(start_cluster): + for node in start_cluster.instances.values(): + node.query( + """ + CREATE TABLE IF NOT EXISTS test_two_level_merge(date Date, zone UInt32, number UInt32) + ENGINE = MergeTree() PARTITION BY toUInt64(number / 1000) ORDER BY tuple(); + + INSERT INTO + test_two_level_merge + SELECT + toDate('2021-09-28') - number / 1000, + 249081628, + number + FROM + numbers(15000); + """ + ) + + # covers only the keys64 method + for node in start_cluster.instances.values(): + print(node.query( + """ + SELECT + throwIf(uniqExact(date) != count(), 'group by is borked') + FROM ( + SELECT + date + FROM + remote('node{1,2}', default.test_two_level_merge) + WHERE + date BETWEEN toDate('2021-09-20') AND toDate('2021-09-28') + AND zone = 249081628 + GROUP by date, zone + ) + SETTINGS + group_by_two_level_threshold = 1, + group_by_two_level_threshold_bytes = 1, + max_threads = 2, + prefer_localhost_replica = 0 + """ + )) From b9f8647353ea9eba64df1fdc782f421d4d737bc5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 6 Oct 2021 11:59:41 +0300 Subject: [PATCH 013/173] Fix build with master --- src/Interpreters/InterpreterKillQueryQuery.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 13108e297f5..df31ed75374 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -355,7 +355,8 @@ BlockIO InterpreterKillQueryQuery::execute() ErrorCodes::ACCESS_DENIED); - res_io.in = std::make_shared(header.cloneWithColumns(std::move(res_columns))); + + res_io.pipeline = QueryPipeline(Pipe(std::make_shared(header.cloneWithColumns(std::move(res_columns))))); break; } From dd067f80cd13d25073d73b5e4433dd85e8a533ae Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Wed, 6 Oct 2021 12:40:24 +0300 Subject: [PATCH 014/173] update RU --- docs/ru/operations/system-tables/replicas.md | 67 +++++++++++--------- 1 file changed, 37 insertions(+), 30 deletions(-) diff --git a/docs/ru/operations/system-tables/replicas.md b/docs/ru/operations/system-tables/replicas.md index ff58355145d..c7bc9c5982e 100644 --- a/docs/ru/operations/system-tables/replicas.md +++ b/docs/ru/operations/system-tables/replicas.md @@ -13,38 +13,43 @@ FORMAT Vertical ``` ``` text +Query id: dc6dcbcb-dc28-4df9-ae27-4354f5b3b13e + Row 1: ────── -database: merge -table: visits -engine: ReplicatedCollapsingMergeTree -is_leader: 1 -can_become_leader: 1 -is_readonly: 0 -is_session_expired: 0 -future_parts: 1 -parts_to_check: 0 -zookeeper_path: /clickhouse/tables/01-06/visits -replica_name: example01-06-1.yandex.ru -replica_path: /clickhouse/tables/01-06/visits/replicas/example01-06-1.yandex.ru -columns_version: 9 -queue_size: 1 -inserts_in_queue: 0 -merges_in_queue: 1 -part_mutations_in_queue: 0 -queue_oldest_time: 2020-02-20 08:34:30 -inserts_oldest_time: 0000-00-00 00:00:00 -merges_oldest_time: 2020-02-20 08:34:30 -part_mutations_oldest_time: 0000-00-00 00:00:00 -oldest_part_to_get: -oldest_part_to_merge_to: 20200220_20284_20840_7 -oldest_part_to_mutate_to: -log_max_index: 596273 -log_pointer: 596274 -last_queue_update: 2020-02-20 08:34:32 -absolute_delay: 0 -total_replicas: 2 -active_replicas: 2 +database: default +table: test +engine: ReplicatedMergeTree +is_leader: 1 +can_become_leader: 1 +is_readonly: 0 +is_session_expired: 0 +future_parts: 0 +parts_to_check: 0 +zookeeper_path: /test/t +replica_name: 1 +replica_path: /test/t/replicas/1 +columns_version: -1 +queue_size: 0 +inserts_in_queue: 0 +merges_in_queue: 0 +part_mutations_in_queue: 0 +queue_oldest_time: 1970-01-01 03:00:00 +inserts_oldest_time: 1970-01-01 03:00:00 +merges_oldest_time: 1970-01-01 03:00:00 +part_mutations_oldest_time: 1970-01-01 03:00:00 +oldest_part_to_get: +oldest_part_to_merge_to: +oldest_part_to_mutate_to: +log_max_index: 0 +log_pointer: 0 +last_queue_update: 1970-01-01 03:00:00 +absolute_delay: 0 +total_replicas: 1 +active_replicas: 1 +last_queue_update_exception: +zookeeper_exception: +replica_is_active: {'1':1} ``` Столбцы: @@ -82,6 +87,8 @@ active_replicas: 2 - `absolute_delay` (`UInt64`) - задержка (в секундах) для текущей реплики. - `total_replicas` (`UInt8`) - общее число известных реплик этой таблицы. - `active_replicas` (`UInt8`) - число реплик этой таблицы, имеющих сессию в ZK; то есть, число работающих реплик. +- `last_queue_update_exception` (`String`) - если в очереди есть битые записи. Особенно важно, когда в ClickHouse нарушается обратная совместимость между версиями, а записи журнала, сделанные более новыми версиями, не могут быть проанализированы старыми версиями. +- `zookeeper_exception` (`String`) - последнее сообщение об исключении. Появляется, если ошибка произошла при получении информации из ZooKeeper. - `replica_is_active` ([Map(String, UInt8)](../../sql-reference/data-types/map.md)) — соответствие между именем реплики и признаком активности реплики. Если запрашивать все столбцы, то таблица может работать слегка медленно, так как на каждую строчку делается несколько чтений из ZK. From ddca7e071292d754cd22a7c22d20bd9c062a45ba Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 6 Oct 2021 12:51:33 +0300 Subject: [PATCH 015/173] Update InterpreterKillQueryQuery.cpp --- src/Interpreters/InterpreterKillQueryQuery.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index df31ed75374..2b949266c17 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -354,8 +354,6 @@ BlockIO InterpreterKillQueryQuery::execute() "Not allowed to kill move partition. To execute this query it's necessary to have the grant " + required_access_rights.toString(), ErrorCodes::ACCESS_DENIED); - - res_io.pipeline = QueryPipeline(Pipe(std::make_shared(header.cloneWithColumns(std::move(res_columns))))); break; From ce645371f5a40862d03b23c973af717255b7620b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Oct 2021 14:46:27 +0300 Subject: [PATCH 016/173] Update container name. --- .../test_backward_compatibility/test_aggregate_fixed_key.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py index 8819be527fd..d895ee0eb78 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py @@ -3,9 +3,9 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', with_zookeeper=True, image='yandex/clickhouse-server', tag='21.3', with_installed_binary=True) -node2 = cluster.add_instance('node2', with_zookeeper=True, image='yandex/clickhouse-server') -node3 = cluster.add_instance('node3', with_zookeeper=True, image='yandex/clickhouse-server') +node1 = cluster.add_instance('node1', with_zookeeper=True, image='clickhouse/server', tag='21.3', with_installed_binary=True) +node2 = cluster.add_instance('node2', with_zookeeper=True, image='clickhouse/server') +node3 = cluster.add_instance('node3', with_zookeeper=True, image='clickhouse/server') @pytest.fixture(scope="module") From e9f3cf665199b358e79c6a3b938a5cb25a33fe46 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 18 Oct 2021 17:49:26 +0300 Subject: [PATCH 017/173] Fix used memory calculation --- src/Common/ProgressIndication.cpp | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index bf3397f50e1..5a3f8cfc350 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -121,11 +121,9 @@ ProgressIndication::MemoryUsage ProgressIndication::getMemoryUsage() const return std::accumulate(thread_data.cbegin(), thread_data.cend(), MemoryUsage{}, [](MemoryUsage const & acc, auto const & host_data) { - auto host_usage = std::accumulate(host_data.second.cbegin(), host_data.second.cend(), ZERO, - [](UInt64 memory, auto const & data) - { - return memory + data.second.memory_usage; - }); + UInt64 host_usage = 0; + if (auto it = host_data.second.find(ZERO); it != host_data.second.end()) + host_usage = it->second.memory_usage; return MemoryUsage{.total = acc.total + host_usage, .max = std::max(acc.max, host_usage)}; }); } From 89fe606d15f94b95b9633525a9dc0e4d052610ff Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 18 Oct 2021 23:16:02 +0300 Subject: [PATCH 018/173] try fix 'some fetches may stuck' --- src/Storages/MergeTree/MergeTreePartInfo.h | 3 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 58 +++-- .../MergeTree/ReplicatedMergeTreeQueue.h | 14 +- .../ReplicatedMergeTreeRestartingThread.cpp | 2 + src/Storages/StorageReplicatedMergeTree.cpp | 216 +++++++++++++----- 5 files changed, 215 insertions(+), 78 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreePartInfo.h b/src/Storages/MergeTree/MergeTreePartInfo.h index 82fe0b860c8..b4c5687209b 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/src/Storages/MergeTree/MergeTreePartInfo.h @@ -139,7 +139,8 @@ struct DetachedPartInfo : public MergeTreePartInfo "clone", "attaching", "deleting", - "tmp-fetch" + "tmp-fetch", + "covered-by-broken", }); /// NOTE: It may parse part info incorrectly. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index f397e73aa31..6c1cd04d4ae 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -55,6 +55,15 @@ void ReplicatedMergeTreeQueue::clear() mutation_pointer.clear(); } +void ReplicatedMergeTreeQueue::setBrokenPartsToEnqueueFetchesOnLoading(Strings && parts_to_fetch) +{ + std::lock_guard lock(state_mutex); + /// Can be called only before queue initialization + assert(broken_parts_to_enqueue_fetches_on_loading.empty()); + assert(virtual_parts.size() == 0); + broken_parts_to_enqueue_fetches_on_loading = std::move(parts_to_fetch); +} + void ReplicatedMergeTreeQueue::initialize(zkutil::ZooKeeperPtr zookeeper) { std::lock_guard lock(state_mutex); @@ -176,6 +185,19 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) } +void ReplicatedMergeTreeQueue::createLogEntriesToFetchBrokenParts() +{ + std::lock_guard lock(state_mutex); + if (broken_parts_to_enqueue_fetches_on_loading.empty()) + return; + + for (const auto & broken_part_name : broken_parts_to_enqueue_fetches_on_loading) + storage.removePartAndEnqueueFetch(broken_part_name); + + broken_parts_to_enqueue_fetches_on_loading.clear(); +} + + void ReplicatedMergeTreeQueue::insertUnlocked( const LogEntryPtr & entry, std::optional & min_unprocessed_insert_time_changed, std::lock_guard & state_lock) @@ -982,7 +1004,7 @@ bool ReplicatedMergeTreeQueue::checkReplaceRangeCanBeRemoved(const MergeTreePart void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( zkutil::ZooKeeperPtr zookeeper, const MergeTreePartInfo & part_info, - const ReplicatedMergeTreeLogEntryData & current) + const std::optional & current) { /// TODO is it possible to simplify it? Queue to_wait; @@ -993,19 +1015,21 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( /// Remove operations with parts, contained in the range to be deleted, from the queue. std::unique_lock lock(state_mutex); - [[maybe_unused]] bool called_from_alter_query_directly = current.replace_range_entry && current.replace_range_entry->columns_version < 0; - assert(currently_executing_drop_or_replace_range || called_from_alter_query_directly); + [[maybe_unused]] bool called_from_alter_query_directly = current && current->replace_range_entry && current->replace_range_entry->columns_version < 0; + [[maybe_unused]] bool called_for_broken_part = !current; + assert(currently_executing_drop_or_replace_range || called_from_alter_query_directly || called_for_broken_part); for (Queue::iterator it = queue.begin(); it != queue.end();) { auto type = (*it)->type; - bool is_simple_producing_op = type == LogEntry::GET_PART || type == LogEntry::ATTACH_PART || type == LogEntry::MERGE_PARTS || type == LogEntry::MUTATE_PART; + bool simple_op_covered = is_simple_producing_op && part_info.contains(MergeTreePartInfo::fromPartName((*it)->new_part_name, format_version)); - if (simple_op_covered || checkReplaceRangeCanBeRemoved(part_info, *it, current)) + bool replace_range_covered = current && checkReplaceRangeCanBeRemoved(part_info, *it, *current); + if (simple_op_covered || replace_range_covered) { if ((*it)->currently_executing) to_wait.push_back(*it); @@ -1035,16 +1059,20 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( } -bool ReplicatedMergeTreeQueue::isNotCoveredByFuturePartsImpl(const String & log_entry_name, const String & new_part_name, +bool ReplicatedMergeTreeQueue::isNotCoveredByFuturePartsImpl(const LogEntry & entry, const String & new_part_name, String & out_reason, std::lock_guard & /* queue_lock */) const { /// Let's check if the same part is now being created by another action. - if (future_parts.count(new_part_name)) + auto entry_for_same_part_it = future_parts.find(new_part_name); + if (entry_for_same_part_it != future_parts.end()) { - const char * format_str = "Not executing log entry {} for part {} " - "because another log entry for the same part is being processed. This shouldn't happen often."; - LOG_INFO(log, format_str, log_entry_name, new_part_name); - out_reason = fmt::format(format_str, log_entry_name, new_part_name); + const LogEntry & another_entry = *entry_for_same_part_it->second; + const char * format_str = "Not executing log entry {} of type {} for part {} " + "because another log entry {} of type {} for the same part ({}) is being processed. This shouldn't happen often."; + LOG_INFO(log, format_str, entry.znode_name, entry.type, entry.new_part_name, + another_entry.znode_name, another_entry.type, another_entry.new_part_name); + out_reason = fmt::format(format_str, entry.znode_name, entry.type, entry.new_part_name, + another_entry.znode_name, another_entry.type, another_entry.new_part_name); return false; /** When the corresponding action is completed, then `isNotCoveredByFuturePart` next time, will succeed, @@ -1067,8 +1095,8 @@ bool ReplicatedMergeTreeQueue::isNotCoveredByFuturePartsImpl(const String & log_ { const char * format_str = "Not executing log entry {} for part {} " "because it is covered by part {} that is currently executing."; - LOG_TRACE(log, format_str, log_entry_name, new_part_name, future_part_elem.first); - out_reason = fmt::format(format_str, log_entry_name, new_part_name, future_part_elem.first); + LOG_TRACE(log, format_str, entry.znode_name, new_part_name, future_part_elem.first); + out_reason = fmt::format(format_str, entry.znode_name, new_part_name, future_part_elem.first); return false; } } @@ -1101,7 +1129,7 @@ bool ReplicatedMergeTreeQueue::addFuturePartIfNotCoveredByThem(const String & pa if (drop_ranges.isAffectedByDropRange(part_name, reject_reason)) return false; - if (isNotCoveredByFuturePartsImpl(entry.znode_name, part_name, reject_reason, lock)) + if (isNotCoveredByFuturePartsImpl(entry, part_name, reject_reason, lock)) { CurrentlyExecuting::setActualPartName(entry, part_name, *this); return true; @@ -1122,7 +1150,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( /// some other entry which is currently executing, then we can postpone this entry. for (const String & new_part_name : entry.getVirtualPartNames(format_version)) { - if (!isNotCoveredByFuturePartsImpl(entry.znode_name, new_part_name, out_postpone_reason, state_lock)) + if (!isNotCoveredByFuturePartsImpl(entry, new_part_name, out_postpone_reason, state_lock)) return false; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 30c8bafcf6e..c059ff7a072 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -156,6 +156,8 @@ private: /// We need it because alters have to be executed sequentially (one by one). ReplicatedMergeTreeAltersSequence alter_sequence; + Strings broken_parts_to_enqueue_fetches_on_loading; + /// List of subscribers /// A subscriber callback is called when an entry queue is deleted mutable std::mutex subscribers_mutex; @@ -208,7 +210,7 @@ private: * Should be called under state_mutex. */ bool isNotCoveredByFuturePartsImpl( - const String & log_entry_name, + const LogEntry & entry, const String & new_part_name, String & out_reason, std::lock_guard & state_lock) const; @@ -324,7 +326,8 @@ public: /** Remove the action from the queue with the parts covered by part_name (from ZK and from the RAM). * And also wait for the completion of their execution, if they are now being executed. */ - void removePartProducingOpsInRange(zkutil::ZooKeeperPtr zookeeper, const MergeTreePartInfo & part_info, const ReplicatedMergeTreeLogEntryData & current); + void removePartProducingOpsInRange(zkutil::ZooKeeperPtr zookeeper, const MergeTreePartInfo & part_info, + const std::optional & current); /** In the case where there are not enough parts to perform the merge in part_name * - move actions with merged parts to the end of the queue @@ -455,6 +458,13 @@ public: /// It's needed because queue itself can trigger it's task handler and in /// this case race condition is possible. QueueLocks lockQueue(); + + /// Can be called only on data parts loading. + /// We need loaded queue to create GET_PART entry for broken (or missing) part, + /// but queue is not loaded yet on data parts loading. + void setBrokenPartsToEnqueueFetchesOnLoading(Strings && parts_to_fetch); + /// Must be called right after queue loading. + void createLogEntriesToFetchBrokenParts(); }; class ReplicatedMergeTreeMergePredicate diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 16597fc4be9..c29ae873c5c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -178,6 +178,8 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() storage.queue.load(zookeeper); + storage.queue.createLogEntriesToFetchBrokenParts(); + /// pullLogsToQueue() after we mark replica 'is_active' (and after we repair if it was lost); /// because cleanup_thread doesn't delete log_pointer of active replicas. storage.queue.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::LOAD); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 416d37cd351..6980d523ed5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1211,44 +1211,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) } /// Add to the queue jobs to pick up the missing parts from other replicas and remove from ZK the information that we have them. - std::vector> exists_futures; - exists_futures.reserve(parts_to_fetch.size()); - for (const String & part_name : parts_to_fetch) - { - String part_path = fs::path(replica_path) / "parts" / part_name; - exists_futures.emplace_back(zookeeper->asyncExists(part_path)); - } - - std::vector> enqueue_futures; - enqueue_futures.reserve(parts_to_fetch.size()); - for (size_t i = 0; i < parts_to_fetch.size(); ++i) - { - const String & part_name = parts_to_fetch[i]; - - Coordination::Requests ops; - - LOG_ERROR(log, "Removing locally missing part from ZooKeeper and queueing a fetch: {}", part_name); - time_t part_create_time = 0; - Coordination::ExistsResponse exists_resp = exists_futures[i].get(); - if (exists_resp.error == Coordination::Error::ZOK) - { - part_create_time = exists_resp.stat.ctime / 1000; - removePartFromZooKeeper(part_name, ops, exists_resp.stat.numChildren > 0); - } - LogEntry log_entry; - log_entry.type = LogEntry::GET_PART; - log_entry.source_replica = ""; - log_entry.new_part_name = part_name; - log_entry.create_time = part_create_time; - - /// We assume that this occurs before the queue is loaded (queue.initialize). - ops.emplace_back(zkutil::makeCreateRequest( - fs::path(replica_path) / "queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); - enqueue_futures.emplace_back(zookeeper->asyncMulti(ops)); - } - - for (auto & future : enqueue_futures) - future.get(); + queue.setBrokenPartsToEnqueueFetchesOnLoading(std::move(parts_to_fetch)); /// Remove extra local parts. for (const DataPartPtr & part : unexpected_parts) @@ -2343,13 +2306,57 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo std::sort(source_queue_names.begin(), source_queue_names.end()); - Strings source_queue; - for (const String & entry_name : source_queue_names) + struct QueueEntryInfo { - String entry; - if (!zookeeper->tryGet(fs::path(source_path) / "queue" / entry_name, entry)) - continue; - source_queue.push_back(entry); + String data; + Coordination::Stat stat; + LogEntryPtr parsed_entry; + bool need_copy = true; + }; + + std::vector source_queue; + ActiveDataPartSet get_part_set{format_version}; + ActiveDataPartSet drop_range_set{format_version}; + + { + std::vector queue_get_futures; + source_queue.resize(source_queue_names.size()); + queue_get_futures.reserve(source_queue_names.size()); + + for (const String & entry_name : source_queue_names) + queue_get_futures.push_back(zookeeper->asyncTryGet(fs::path(source_path) / "queue" / entry_name)); + for (size_t i = 0; i < source_queue_names.size(); ++i) + { + auto res = queue_get_futures[i].get(); + /// It's ok if entry is already executed and removed: we also will get source parts set. + if (res.error == Coordination::Error::ZNONODE) + continue; + + assert(res.error == Coordination::Error::ZOK); + auto & info = source_queue[i]; + info.data = std::move(res.data); + info.stat = std::move(res.stat); + try + { + info.parsed_entry = LogEntry::parse(info.data, info.stat); + } + catch (...) + { + tryLogCurrentException(log, "Cannot parse source queue entry " + source_queue_names[i]); + } + + /// It may be ok if source replica has newer version. We will copy entry as is. + if (!info.parsed_entry) + continue; + + info.parsed_entry->znode_name = source_queue_names[i]; + + if (info.parsed_entry->type == LogEntry::GET_PART) + get_part_set.add(info.parsed_entry->new_part_name); + + if (info.parsed_entry->type == LogEntry::DROP_RANGE) + drop_range_set.add(info.parsed_entry->new_part_name); + } } /// We should do it after copying queue, because some ALTER_METADATA entries can be lost otherwise. @@ -2357,9 +2364,10 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo /// Add to the queue jobs to receive all the active parts that the reference/master replica has. Strings source_replica_parts = zookeeper->getChildren(fs::path(source_path) / "parts"); - ActiveDataPartSet active_parts_set(format_version, source_replica_parts); + for (const auto & active_part : source_replica_parts) + get_part_set.add(active_part); - Strings active_parts = active_parts_set.getParts(); + Strings active_parts = get_part_set.getParts(); /// Remove local parts if source replica does not have them, because such parts will never be fetched by other replicas. Strings local_parts_in_zk = zookeeper->getChildren(fs::path(replica_path) / "parts"); @@ -2367,7 +2375,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo for (const auto & part : local_parts_in_zk) { - if (active_parts_set.getContainingPart(part).empty()) + if (get_part_set.getContainingPart(part).empty()) { parts_to_remove_from_zk.emplace_back(part); LOG_WARNING(log, "Source replica does not have part {}. Removing it from ZooKeeper.", part); @@ -2392,7 +2400,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo for (const auto & part : local_active_parts) { - if (active_parts_set.getContainingPart(part->name).empty()) + if (get_part_set.getContainingPart(part->name).empty()) { parts_to_remove_from_working_set.emplace_back(part); LOG_WARNING(log, "Source replica does not have part {}. Removing it from working set.", part->name); @@ -2412,13 +2420,54 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo removePartsFromWorkingSet(parts_to_remove_from_working_set, true); + std::unordered_set created_get_parts; + + /// Avoid creation of GET_PART entries which covered by another GET_PART or DROP_RANGE + /// and creation of multiple entries with the same new_part_name. + auto should_ignore_log_entry = [&] (const String & part_name, const String & log_msg_context) -> bool + { + /// We should not create entries covered by DROP_RANGE, because we will remove them anyway (kind of optimization). + String covering_drop_range = drop_range_set.getContainingPart(part_name); + if (!covering_drop_range.empty()) + { + LOG_TRACE(log, "{} {}: it's covered by DROP_RANGE {}", log_msg_context, part_name, covering_drop_range); + return true; + } + + /// We should not create entries covered by GET_PART, + /// because GET_PART entry has no source parts and we can execute it only by fetching. + /// Parts covered by GET_PART are useless and may cause replication to stuck if covered part is lost. + String covering_get_part_entry = get_part_set.getContainingPart(part_name); + + if (covering_get_part_entry.empty()) + return false; + + if (covering_get_part_entry != part_name) + { + LOG_TRACE(log, "{} {}: it's covered by GET_PART {}", log_msg_context, part_name, covering_get_part_entry); + return true; + } + + if (created_get_parts.count(part_name)) + { + /// NOTE It would be better to copy log entry instead of creating GET_PART + /// if there are GET_PART and log entry of other type with the same new_part_name. + /// But it's a bit harder to implement, because it requires full-fledged virtual_parts set. + LOG_TRACE(log, "{} {}: GET_PART for it is already created", log_msg_context, part_name); + return true; + } + + return false; + }; + for (const String & name : active_parts) { + if (should_ignore_log_entry(name, "Not fetching")) + continue; + LogEntry log_entry; - if (!are_restoring_replica) - log_entry.type = LogEntry::GET_PART; - else + if (are_restoring_replica) { LOG_DEBUG(log, "Obtaining checksum for path {}", name); @@ -2443,23 +2492,46 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo const auto [lo, hi] = desired_checksums.hash_of_all_files; log_entry.part_checksum = getHexUIntUppercase(hi) + getHexUIntUppercase(lo); } + else + { + log_entry.type = LogEntry::GET_PART; + } log_entry.source_replica = ""; log_entry.new_part_name = name; log_entry.create_time = tryGetPartCreateTime(zookeeper, source_path, name); + LOG_TEST(log, "Enqueueing {} for fetch", name); zookeeper->create(fs::path(replica_path) / "queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential); + created_get_parts.insert(name); } - LOG_DEBUG(log, "Queued {} parts to be fetched", active_parts.size()); + size_t total_parts_to_fetch = created_get_parts.size(); + LOG_DEBUG(log, "Queued {} parts to be fetched, {} parts ignored", total_parts_to_fetch, active_parts.size() - total_parts_to_fetch); /// Add content of the reference/master replica queue to the queue. - for (const String & entry : source_queue) + size_t total_entries_to_copy = 0; + for (const auto & entry_info : source_queue) { - zookeeper->create(fs::path(replica_path) / "queue/queue-", entry, zkutil::CreateMode::PersistentSequential); + if (entry_info.parsed_entry && !entry_info.parsed_entry->new_part_name.empty()) + { + const String & part_name = entry_info.parsed_entry->new_part_name; + const String & entry_name = entry_info.parsed_entry->znode_name; + const auto & entry_type = entry_info.parsed_entry->type; + + if (should_ignore_log_entry(part_name, fmt::format("Not copying {} {} ", entry_name, entry_type))) + continue; + + if (entry_info.parsed_entry->type == LogEntry::GET_PART) + created_get_parts.insert(part_name); + } + + LOG_TEST(log, "Copying entry {}", entry_info.data); + zookeeper->create(fs::path(replica_path) / "queue/queue-", entry_info.data, zkutil::CreateMode::PersistentSequential); + ++total_entries_to_copy; } - LOG_DEBUG(log, "Copied {} queue entries", source_queue.size()); + LOG_DEBUG(log, "Copied {} queue entries, {} entries ignored", total_entries_to_copy, source_queue.size() - total_entries_to_copy); } @@ -3196,6 +3268,34 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n { auto zookeeper = getZooKeeper(); + /// We don't know exactly what happened to broken part + /// and we are going to remove all covered log entries. + /// It's quite dangerous, so clone covered parts to detached. + auto broken_part_info = MergeTreePartInfo::fromPartName(part_name, format_version); + + auto partition_range = getDataPartsPartitionRange(broken_part_info.partition_id); + for (const auto & part : partition_range) + { + if (!broken_part_info.contains(part->info)) + continue; + + /// Broken part itself ether already moved to detached or does not exist. + assert(broken_part_info != part->info); + part->makeCloneInDetached("covered-by-broken", getInMemoryMetadataPtr()); + } + + /// It's possible that queue contains entries covered by part_name. + /// For example, we had GET_PART all_1_42_5 and MUTATE_PART all_1_42_5_63, + /// then all_1_42_5_63 was executed by fetching, but part was written to disk incorrectly. + /// In this case we have to remove it as broken and create GET_PART all_1_42_5_63 to fetch it again, + /// but GET_PART all_1_42_5 may be still in the queue. + /// We should remove all covered entries before creating GET_PART entry, because: + /// 1. In the situation described above, we do not know how to merge/mutate all_1_42_5_63 from all_1_42_5, + /// so GET_PART all_1_42_5 (and all source parts) is useless. The only thing we can do is to fetch all_1_42_5_63. + /// 2. If all_1_42_5_63 is lost, then replication may stuck waiting for all_1_42_5_63 to appear, + /// because we may have some covered parts (more precisely, parts with the same min and max blocks) + queue.removePartProducingOpsInRange(zookeeper, broken_part_info, {}); + String part_path = fs::path(replica_path) / "parts" / part_name; Coordination::Requests ops; @@ -3218,11 +3318,7 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n fs::path(replica_path) / "queue/queue-", log_entry->toString(), zkutil::CreateMode::PersistentSequential)); - auto results = zookeeper->multi(ops); - - String path_created = dynamic_cast(*results.back()).path_created; - log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1); - queue.insert(zookeeper, log_entry); + zookeeper->multi(ops); } From caf972db48c38a24c588c5c333016979812ed6b5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 19 Oct 2021 14:56:04 +0300 Subject: [PATCH 019/173] fix --- src/Storages/StorageReplicatedMergeTree.cpp | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6980d523ed5..9a3c88ca83f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2308,10 +2308,9 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo struct QueueEntryInfo { - String data; - Coordination::Stat stat; - LogEntryPtr parsed_entry; - bool need_copy = true; + String data = {}; + Coordination::Stat stat = {}; + LogEntryPtr parsed_entry = {}; }; std::vector source_queue; @@ -2320,11 +2319,12 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo { std::vector queue_get_futures; - source_queue.resize(source_queue_names.size()); queue_get_futures.reserve(source_queue_names.size()); for (const String & entry_name : source_queue_names) queue_get_futures.push_back(zookeeper->asyncTryGet(fs::path(source_path) / "queue" / entry_name)); + + source_queue.reserve(source_queue_names.size()); for (size_t i = 0; i < source_queue_names.size(); ++i) { auto res = queue_get_futures[i].get(); @@ -2333,7 +2333,8 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo continue; assert(res.error == Coordination::Error::ZOK); - auto & info = source_queue[i]; + source_queue.emplace_back(); + auto & info = source_queue.back(); info.data = std::move(res.data); info.stat = std::move(res.stat); try @@ -2448,6 +2449,11 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo return true; } + /// NOTE: It does not completely avoids duplication of GET_PART entries, + /// because it's possible that source replica has executed some GET_PART after we copied it's queue, + /// but before we copied its active parts set. In this case we will GET_PART entry in our queue + /// and later will pull the original GET_PART from replication log. + /// It should not cause any issues, but it does not allow to get rid of duplicated entries and add an assertion. if (created_get_parts.count(part_name)) { /// NOTE It would be better to copy log entry instead of creating GET_PART @@ -2513,6 +2519,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo size_t total_entries_to_copy = 0; for (const auto & entry_info : source_queue) { + assert(!entry_info.data.empty()); if (entry_info.parsed_entry && !entry_info.parsed_entry->new_part_name.empty()) { const String & part_name = entry_info.parsed_entry->new_part_name; From 170a4957ca28862fdf1915c9236b97872a5f0fe2 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 20 Oct 2021 01:31:39 +0300 Subject: [PATCH 020/173] Add normalizeUTF8NFC function --- .../functions/string-functions.md | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 05c291bb41d..6c5ed485a3d 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -810,6 +810,28 @@ Result: └─────┘ ``` +## normalizeUTF8NFC {#normalizeutf8nfc} + +Converts a string to [NFC normalized form](https://en.wikipedia.org/wiki/Unicode_equivalence#Normal_forms), assuming the string contains a set of bytes that make up a UTF-8 encoded text. + +**Syntax** + +``` sql +normalizeUTF8NFKD(words) +``` + +**Arguments** + +- `words` — Input string that contains UTF-8 encoded text. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- NFC normalized string. + +Type: [String](../../sql-reference/data-types/string.md). + + + ## encodeXMLComponent {#encode-xml-component} Escapes characters to place string into XML text node or attribute. From 73f65bbe75db733afb880421f222e95d75c8461f Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 20 Oct 2021 01:42:39 +0300 Subject: [PATCH 021/173] Rewrite Normalization Form C (NFC) --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 6c5ed485a3d..4ac3a175aba 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -826,7 +826,7 @@ normalizeUTF8NFKD(words) **Returned value** -- NFC normalized string. +- String transformed to normalization form C (NFC). Type: [String](../../sql-reference/data-types/string.md). From 6b34398e9cd99c7f0b925342356b368d6d17e884 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 20 Oct 2021 17:17:20 +0300 Subject: [PATCH 022/173] support VALUES format in async inserts --- src/Parsers/ParserInsertQuery.cpp | 9 +- src/Processors/Formats/IInputFormat.h | 3 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 146 ++++++++++-------- .../Formats/Impl/ValuesBlockInputFormat.h | 6 +- .../getSourceFromASTInsertQuery.cpp | 8 +- .../02015_async_insert_7.reference | 6 + .../0_stateless/02015_async_insert_7.sh | 20 +++ 7 files changed, 126 insertions(+), 72 deletions(-) create mode 100644 tests/queries/0_stateless/02015_async_insert_7.reference create mode 100755 tests/queries/0_stateless/02015_async_insert_7.sh diff --git a/src/Parsers/ParserInsertQuery.cpp b/src/Parsers/ParserInsertQuery.cpp index 6ac6d3b50b9..1bc552af384 100644 --- a/src/Parsers/ParserInsertQuery.cpp +++ b/src/Parsers/ParserInsertQuery.cpp @@ -131,18 +131,22 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } Pos before_values = pos; + String format_str; /// VALUES or FROM INFILE or FORMAT or SELECT if (!infile && s_values.ignore(pos, expected)) { /// If VALUES is defined in query, everything except setting will be parsed as data data = pos->begin; + format_str = "Values"; } else if (s_format.ignore(pos, expected)) { /// If FORMAT is defined, read format name if (!name_p.parse(pos, format, expected)) return false; + + tryGetIdentifierNameInto(format, format_str); } else if (s_select.ignore(pos, expected) || s_with.ignore(pos,expected)) { @@ -155,6 +159,8 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) /// FORMAT section is expected if we have input() in SELECT part if (s_format.ignore(pos, expected) && !name_p.parse(pos, format, expected)) return false; + + tryGetIdentifierNameInto(format, format_str); } else if (s_watch.ignore(pos, expected)) { @@ -242,9 +248,8 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) tryGetIdentifierNameInto(table, query->table_id.table_name); } - tryGetIdentifierNameInto(format, query->format); - query->columns = columns; + query->format = std::move(format_str); query->select = select; query->watch = watch; query->settings_ast = settings_ast; diff --git a/src/Processors/Formats/IInputFormat.h b/src/Processors/Formats/IInputFormat.h index f133161c3ec..8f5992efb1b 100644 --- a/src/Processors/Formats/IInputFormat.h +++ b/src/Processors/Formats/IInputFormat.h @@ -55,6 +55,8 @@ public: */ virtual void resetParser(); + virtual void setReadBuffer(ReadBuffer & in_); + virtual const BlockMissingValues & getMissingValues() const { static const BlockMissingValues none; @@ -70,7 +72,6 @@ public: void setCurrentUnitNumber(size_t current_unit_number_) { current_unit_number = current_unit_number_; } void addBuffer(std::unique_ptr buffer) { owned_buffers.emplace_back(std::move(buffer)); } - void setReadBuffer(ReadBuffer & in_); protected: ColumnMappingPtr column_mapping{}; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 0f6a21055d0..c15d0d608ee 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -32,13 +32,25 @@ namespace ErrorCodes } -ValuesBlockInputFormat::ValuesBlockInputFormat(ReadBuffer & in_, const Block & header_, const RowInputFormatParams & params_, - const FormatSettings & format_settings_) - : IInputFormat(header_, buf), buf(in_), params(params_), - format_settings(format_settings_), num_columns(header_.columns()), - parser_type_for_column(num_columns, ParserType::Streaming), - attempts_to_deduce_template(num_columns), attempts_to_deduce_template_cached(num_columns), - rows_parsed_using_template(num_columns), templates(num_columns), types(header_.getDataTypes()) +ValuesBlockInputFormat::ValuesBlockInputFormat( + ReadBuffer & in_, + const Block & header_, + const RowInputFormatParams & params_, + const FormatSettings & format_settings_) + : ValuesBlockInputFormat(std::make_unique(in_), header_, params_, format_settings_) +{ +} + +ValuesBlockInputFormat::ValuesBlockInputFormat( + std::unique_ptr buf_, + const Block & header_, + const RowInputFormatParams & params_, + const FormatSettings & format_settings_) + : IInputFormat(header_, *buf_), buf(std::move(buf_)), + params(params_), format_settings(format_settings_), num_columns(header_.columns()), + parser_type_for_column(num_columns, ParserType::Streaming), + attempts_to_deduce_template(num_columns), attempts_to_deduce_template_cached(num_columns), + rows_parsed_using_template(num_columns), templates(num_columns), types(header_.getDataTypes()) { serializations.resize(types.size()); for (size_t i = 0; i < types.size(); ++i) @@ -58,8 +70,8 @@ Chunk ValuesBlockInputFormat::generate() { try { - skipWhitespaceIfAny(buf); - if (buf.eof() || *buf.position() == ';') + skipWhitespaceIfAny(*buf); + if (buf->eof() || *buf->position() == ';') break; readRow(columns, rows_in_block); } @@ -99,12 +111,12 @@ Chunk ValuesBlockInputFormat::generate() void ValuesBlockInputFormat::readRow(MutableColumns & columns, size_t row_num) { - assertChar('(', buf); + assertChar('(', *buf); for (size_t column_idx = 0; column_idx < num_columns; ++column_idx) { - skipWhitespaceIfAny(buf); - PeekableReadBufferCheckpoint checkpoint{buf}; + skipWhitespaceIfAny(*buf); + PeekableReadBufferCheckpoint checkpoint{*buf}; bool read; /// Parse value using fast streaming parser for literals and slow SQL parser for expressions. @@ -123,9 +135,9 @@ void ValuesBlockInputFormat::readRow(MutableColumns & columns, size_t row_num) /// If read is true, value still may be missing. Bit mask for these values will be copied from ConstantExpressionTemplate later. } - skipWhitespaceIfAny(buf); - if (!buf.eof() && *buf.position() == ',') - ++buf.position(); + skipWhitespaceIfAny(*buf); + if (!buf->eof() && *buf->position() == ',') + ++buf->position(); ++total_rows; } @@ -134,7 +146,7 @@ bool ValuesBlockInputFormat::tryParseExpressionUsingTemplate(MutableColumnPtr & { /// Try to parse expression using template if one was successfully deduced while parsing the first row auto settings = context->getSettingsRef(); - if (templates[column_idx]->parseExpression(buf, format_settings, settings)) + if (templates[column_idx]->parseExpression(*buf, format_settings, settings)) { ++rows_parsed_using_template[column_idx]; return true; @@ -154,7 +166,7 @@ bool ValuesBlockInputFormat::tryParseExpressionUsingTemplate(MutableColumnPtr & } /// Do not use this template anymore templates[column_idx].reset(); - buf.rollbackToCheckpoint(); + buf->rollbackToCheckpoint(); /// It will deduce new template or fallback to slow SQL parser return parseExpression(*column, column_idx); @@ -169,13 +181,13 @@ bool ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx) const auto & type = types[column_idx]; const auto & serialization = serializations[column_idx]; if (format_settings.null_as_default && !type->isNullable()) - read = SerializationNullable::deserializeTextQuotedImpl(column, buf, format_settings, serialization); + read = SerializationNullable::deserializeTextQuotedImpl(column, *buf, format_settings, serialization); else - serialization->deserializeTextQuoted(column, buf, format_settings); + serialization->deserializeTextQuoted(column, *buf, format_settings); rollback_on_exception = true; - skipWhitespaceIfAny(buf); + skipWhitespaceIfAny(*buf); assertDelimiterAfterValue(column_idx); return read; } @@ -190,7 +202,7 @@ bool ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx) /// Switch to SQL parser and don't try to use streaming parser for complex expressions /// Note: Throwing exceptions for each expression may be very slow because of stacktraces - buf.rollbackToCheckpoint(); + buf->rollbackToCheckpoint(); return parseExpression(column, column_idx); } } @@ -284,11 +296,11 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx /// We need continuous memory containing the expression to use Lexer skipToNextRow(0, 1); - buf.makeContinuousMemoryFromCheckpointToPos(); - buf.rollbackToCheckpoint(); + buf->makeContinuousMemoryFromCheckpointToPos(); + buf->rollbackToCheckpoint(); Expected expected; - Tokens tokens(buf.position(), buf.buffer().end()); + Tokens tokens(buf->position(), buf->buffer().end()); IParser::Pos token_iterator(tokens, settings.max_parser_depth); ASTPtr ast; @@ -302,7 +314,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx if (!parsed) throw Exception("Cannot parse expression of type " + type.getName() + " here: " - + String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position())), + + String(buf->position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf->buffer().end() - buf->position())), ErrorCodes::SYNTAX_ERROR); ++token_iterator; @@ -316,9 +328,9 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx try { const auto & serialization = serializations[column_idx]; - serialization->deserializeTextQuoted(column, buf, format_settings); + serialization->deserializeTextQuoted(column, *buf, format_settings); rollback_on_exception = true; - skipWhitespaceIfAny(buf); + skipWhitespaceIfAny(*buf); if (checkDelimiterAfterValue(column_idx)) ok = true; } @@ -366,8 +378,8 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx else ++attempts_to_deduce_template[column_idx]; - buf.rollbackToCheckpoint(); - if (templates[column_idx]->parseExpression(buf, format_settings, settings)) + buf->rollbackToCheckpoint(); + if (templates[column_idx]->parseExpression(*buf, format_settings, settings)) { ++rows_parsed_using_template[column_idx]; parser_type_for_column[column_idx] = ParserType::BatchTemplate; @@ -384,9 +396,9 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx std::rethrow_exception(exception); else { - buf.rollbackToCheckpoint(); - size_t len = const_cast(token_iterator->begin) - buf.position(); - throw Exception("Cannot deduce template of expression: " + std::string(buf.position(), len), ErrorCodes::SYNTAX_ERROR); + buf->rollbackToCheckpoint(); + size_t len = const_cast(token_iterator->begin) - buf->position(); + throw Exception("Cannot deduce template of expression: " + std::string(buf->position(), len), ErrorCodes::SYNTAX_ERROR); } } /// Continue parsing without template @@ -397,7 +409,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx throw Exception("Interpreting expressions is disabled", ErrorCodes::SUPPORT_IS_DISABLED); /// Try to evaluate single expression if other parsers don't work - buf.position() = const_cast(token_iterator->begin); + buf->position() = const_cast(token_iterator->begin); std::pair value_raw = evaluateConstantExpression(ast, context); @@ -416,10 +428,10 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx type.insertDefaultInto(column); return false; } - buf.rollbackToCheckpoint(); + buf->rollbackToCheckpoint(); throw Exception{"Cannot insert NULL value into a column of type '" + type.getName() + "'" + " at: " + - String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position())), + String(buf->position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf->buffer().end() - buf->position())), ErrorCodes::TYPE_MISMATCH}; } @@ -430,61 +442,61 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx /// Can be used in fileSegmentationEngine for parallel parsing of Values bool ValuesBlockInputFormat::skipToNextRow(size_t min_chunk_bytes, int balance) { - skipWhitespaceIfAny(buf); - if (buf.eof() || *buf.position() == ';') + skipWhitespaceIfAny(*buf); + if (buf->eof() || *buf->position() == ';') return false; bool quoted = false; - size_t chunk_begin_buf_count = buf.count(); - while (!buf.eof() && (balance || buf.count() - chunk_begin_buf_count < min_chunk_bytes)) + size_t chunk_begin_buf_count = buf->count(); + while (!buf->eof() && (balance || buf->count() - chunk_begin_buf_count < min_chunk_bytes)) { - buf.position() = find_first_symbols<'\\', '\'', ')', '('>(buf.position(), buf.buffer().end()); - if (buf.position() == buf.buffer().end()) + buf->position() = find_first_symbols<'\\', '\'', ')', '('>(buf->position(), buf->buffer().end()); + if (buf->position() == buf->buffer().end()) continue; - if (*buf.position() == '\\') + if (*buf->position() == '\\') { - ++buf.position(); - if (!buf.eof()) - ++buf.position(); + ++buf->position(); + if (!buf->eof()) + ++buf->position(); } - else if (*buf.position() == '\'') + else if (*buf->position() == '\'') { quoted ^= true; - ++buf.position(); + ++buf->position(); } - else if (*buf.position() == ')') + else if (*buf->position() == ')') { - ++buf.position(); + ++buf->position(); if (!quoted) --balance; } - else if (*buf.position() == '(') + else if (*buf->position() == '(') { - ++buf.position(); + ++buf->position(); if (!quoted) ++balance; } } - if (!buf.eof() && *buf.position() == ',') - ++buf.position(); + if (!buf->eof() && *buf->position() == ',') + ++buf->position(); return true; } void ValuesBlockInputFormat::assertDelimiterAfterValue(size_t column_idx) { if (unlikely(!checkDelimiterAfterValue(column_idx))) - throwAtAssertionFailed((column_idx + 1 == num_columns) ? ")" : ",", buf); + throwAtAssertionFailed((column_idx + 1 == num_columns) ? ")" : ",", *buf); } bool ValuesBlockInputFormat::checkDelimiterAfterValue(size_t column_idx) { - skipWhitespaceIfAny(buf); + skipWhitespaceIfAny(*buf); if (likely(column_idx + 1 != num_columns)) - return checkChar(',', buf); + return checkChar(',', *buf); else - return checkChar(')', buf); + return checkChar(')', *buf); } bool ValuesBlockInputFormat::shouldDeduceNewTemplate(size_t column_idx) @@ -516,21 +528,21 @@ bool ValuesBlockInputFormat::shouldDeduceNewTemplate(size_t column_idx) void ValuesBlockInputFormat::readPrefix() { /// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it. - skipBOMIfExists(buf); + skipBOMIfExists(*buf); } void ValuesBlockInputFormat::readSuffix() { - if (!buf.eof() && *buf.position() == ';') + if (!buf->eof() && *buf->position() == ';') { - ++buf.position(); - skipWhitespaceIfAny(buf); - if (buf.hasUnreadData()) + ++buf->position(); + skipWhitespaceIfAny(*buf); + if (buf->hasUnreadData()) throw Exception("Cannot read data after semicolon", ErrorCodes::CANNOT_READ_ALL_DATA); return; } - if (buf.hasUnreadData()) + if (buf->hasUnreadData()) throw Exception("Unread data in PeekableReadBuffer will be lost. Most likely it's a bug.", ErrorCodes::LOGICAL_ERROR); } @@ -539,10 +551,16 @@ void ValuesBlockInputFormat::resetParser() IInputFormat::resetParser(); // I'm not resetting parser modes here. // There is a good chance that all messages have the same format. - buf.reset(); + buf->reset(); total_rows = 0; } +void ValuesBlockInputFormat::setReadBuffer(ReadBuffer & in_) +{ + buf = std::make_unique(in_); + IInputFormat::setReadBuffer(*buf); +} + void registerInputFormatValues(FormatFactory & factory) { factory.registerInputFormat("Values", []( diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index b1e44f5d104..5bbd4bea5ba 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -32,6 +32,7 @@ public: String getName() const override { return "ValuesBlockInputFormat"; } void resetParser() override; + void setReadBuffer(ReadBuffer & in_) override; /// TODO: remove context somehow. void setContext(ContextPtr context_) { context = Context::createCopy(context_); } @@ -39,6 +40,9 @@ public: const BlockMissingValues & getMissingValues() const override { return block_missing_values; } private: + ValuesBlockInputFormat(std::unique_ptr buf_, const Block & header_, const RowInputFormatParams & params_, + const FormatSettings & format_settings_); + enum class ParserType { Streaming, @@ -66,7 +70,7 @@ private: bool skipToNextRow(size_t min_chunk_bytes = 0, int balance = 0); - PeekableReadBuffer buf; + std::unique_ptr buf; const RowInputFormatParams params; diff --git a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp index 86998614189..64c8a01bb9c 100644 --- a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp +++ b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp @@ -42,12 +42,12 @@ InputFormatPtr getInputFormatFromASTInsertQuery( if (ast_insert_query->infile && context->getApplicationType() == Context::ApplicationType::SERVER) throw Exception("Query has infile and was send directly to server", ErrorCodes::UNKNOWN_TYPE_OF_QUERY); - String format = ast_insert_query->format; - if (format.empty()) + if (ast_insert_query->format.empty()) { if (input_function) throw Exception("FORMAT must be specified for function input()", ErrorCodes::INVALID_USAGE_OF_INPUT); - format = "Values"; + else + throw Exception("Logical error: INSERT query requires format to be set", ErrorCodes::LOGICAL_ERROR); } /// Data could be in parsed (ast_insert_query.data) and in not parsed yet (input_buffer_tail_part) part of query. @@ -59,7 +59,7 @@ InputFormatPtr getInputFormatFromASTInsertQuery( : std::make_unique(); /// Create a source from input buffer using format from query - auto source = context->getInputFormat(format, *input_buffer, header, context->getSettings().max_insert_block_size); + auto source = context->getInputFormat(ast_insert_query->format, *input_buffer, header, context->getSettings().max_insert_block_size); source->addBuffer(std::move(input_buffer)); return source; } diff --git a/tests/queries/0_stateless/02015_async_insert_7.reference b/tests/queries/0_stateless/02015_async_insert_7.reference new file mode 100644 index 00000000000..8816e320ee8 --- /dev/null +++ b/tests/queries/0_stateless/02015_async_insert_7.reference @@ -0,0 +1,6 @@ +1 a +2 b +3 c +4 d +5 e +6 f diff --git a/tests/queries/0_stateless/02015_async_insert_7.sh b/tests/queries/0_stateless/02015_async_insert_7.sh new file mode 100755 index 00000000000..c8cbbc48a29 --- /dev/null +++ b/tests/queries/0_stateless/02015_async_insert_7.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=1" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE async_inserts (id UInt32, s String) ENGINE = Memory" + +${CLICKHOUSE_CURL} -sS $url -d "INSERT INTO async_inserts VALUES (1, 'a') (2, 'b')" & +${CLICKHOUSE_CURL} -sS $url -d "INSERT INTO async_inserts VALUES (3, 'c'), (4, 'd')" & +${CLICKHOUSE_CURL} -sS $url -d "INSERT INTO async_inserts VALUES (5, 'e'), (6, 'f'), " & + +wait + +${CLICKHOUSE_CLIENT} -q "SELECT * FROM async_inserts ORDER BY id" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE async_inserts" From 1241db37950503f9eb72b72cf2172ae17fc8b620 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Wed, 20 Oct 2021 20:18:14 +0300 Subject: [PATCH 023/173] document the SQL-statements INTERSECT, EXCEPT and ANY, ALL operators --- docs/en/sql-reference/operators/index.md | 48 ++++++++++ .../sql-reference/statements/select/except.md | 86 ++++++++++++++++++ .../sql-reference/statements/select/index.md | 2 + .../statements/select/intersect.md | 87 +++++++++++++++++++ 4 files changed, 223 insertions(+) create mode 100644 docs/en/sql-reference/statements/select/except.md create mode 100644 docs/en/sql-reference/statements/select/intersect.md diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index dbda5f5dd69..159fc692e94 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -71,6 +71,54 @@ ClickHouse transforms operators to their corresponding functions at the query pa `a GLOBAL NOT IN ...` – The `globalNotIn(a, b)` function. +`ALL` – The operator allows you to get data by comparing the value with the list of values returned by the subquery. The comparison condition must be running for all values of the subquery. + +The subquery must select values of the same type as those compared in the main predicate. + +**Example** + +Query: + +``` sql +SELECT number AS a FROM numbers(10) WHERE a > all(SELECT number FROM numbers(3, 3)); +``` + +Result: + +``` text +┌─a─┐ +│ 6 │ +│ 7 │ +│ 8 │ +│ 9 │ +└───┘ +``` + +`ANY` – The operator is comparing a value to a set of values returned by a subquery. The comparison condition must be running for any values of the subquery. + +The subquery must select values of the same type as those compared in the main predicate. + +**Example** + +Query: + +``` sql +SELECT number AS a FROM numbers(10) WHERE a > any(SELECT number FROM numbers(3, 3)); +``` + +Result: + +``` text +┌─a─┐ +│ 4 │ +│ 5 │ +│ 6 │ +│ 7 │ +│ 8 │ +│ 9 │ +└───┘ +``` + ## Operators for Working with Dates and Times {#operators-datetime} ### EXTRACT {#operator-extract} diff --git a/docs/en/sql-reference/statements/select/except.md b/docs/en/sql-reference/statements/select/except.md new file mode 100644 index 00000000000..ffbc0fae3da --- /dev/null +++ b/docs/en/sql-reference/statements/select/except.md @@ -0,0 +1,86 @@ +--- +toc_title: EXCEPT +--- + +# EXCEPT Clause {#except-clause} + +The `EXCEPT` clause returns only those rows that result from the first query without the second. The queries must match the number of columns, order, and type. The result of `EXCEPT` can contain duplicate rows. + +Multiple `EXCEPT` statements execute left to right if brackets are not specified. The `EXCEPT` operator has the same priority as the `UNION` clause and lower priority as the `INTERSECT` clause. + +``` sql +SELECT column1 [, column2 ] +FROM table1 +[WHERE condition] + +EXCEPT + +SELECT column1 [, column2 ] +FROM table2 +[WHERE condition] + +``` +The condition could be any expression based on your requirements. + +**Example** + +Query: + +``` sql +SELECT number FROM numbers(1,10) EXCEPT SELECT number FROM numbers(3,6); +``` + +Result: + +``` text +┌─number─┐ +│ 1 │ +│ 2 │ +│ 9 │ +│ 10 │ +└────────┘ +``` + +Query: + +``` sql +SELECT number FROM numbers(1,10) WHERE number > 5 EXCEPT SELECT number FROM numbers(3,6) WHERE number < 7; +``` + +Result: + +``` text +┌─number─┐ +│ 7 │ +│ 8 │ +│ 9 │ +│ 10 │ +└────────┘ +``` + +Query: + +``` sql +CREATE TABLE t1(one String, two String, three String) ENGINE=Memory(); +CREATE TABLE t2(four String, five String, six String) ENGINE=Memory(); + +INSERT INTO t1 VALUES ('q', 'm', 'b'), ('s', 'd', 'f'), ('l', 'p', 'o'), ('s', 'd', 'f'), ('s', 'd', 'f'), ('k', 't', 'd'), ('l', 'p', 'o'); +INSERT INTO t2 VALUES ('q', 'm', 'b'), ('b', 'd', 'k'), ('s', 'y', 't'), ('s', 'd', 'f'), ('m', 'f', 'o'), ('k', 'k', 'd'); + +SELECT * FROM t1 EXCEPT SELECT * FROM t2; +``` + +Result: + +``` text +┌─one─┬─two─┬─three─┐ +│ l │ p │ o │ +│ k │ t │ d │ +│ l │ p │ o │ +└─────┴─────┴───────┘ +``` + +**See Also** + +- [UNION](union.md#union-clause) +- [INTERSECT](except.md#except-clause) \ No newline at end of file diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index 0427764475a..39d12f880e4 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -49,6 +49,8 @@ Specifics of each optional clause are covered in separate sections, which are li - [LIMIT clause](../../../sql-reference/statements/select/limit.md) - [OFFSET clause](../../../sql-reference/statements/select/offset.md) - [UNION clause](../../../sql-reference/statements/select/union.md) +- [INTERSECT clause](../../../sql-reference/statements/select/intersect.md) +- [EXCEPT clause](../../../sql-reference/statements/select/except.md) - [INTO OUTFILE clause](../../../sql-reference/statements/select/into-outfile.md) - [FORMAT clause](../../../sql-reference/statements/select/format.md) diff --git a/docs/en/sql-reference/statements/select/intersect.md b/docs/en/sql-reference/statements/select/intersect.md new file mode 100644 index 00000000000..89446c68e62 --- /dev/null +++ b/docs/en/sql-reference/statements/select/intersect.md @@ -0,0 +1,87 @@ +--- +toc_title: INTERSECT +--- + +# INTERSECT Clause {#intersect-clause} + +The `INTERSECT` clause returns only those rows that result from first and second queries. The queries must match the number of columns, order, type, and names. The result of `INTERSECT` can contain duplicate rows. + +Multiple `INTERSECT` statements execute left to right if brackets are not specified. The `INTERSECT` operator has a higher priority than the `UNION` and `EXCEPT` clause. + + +``` sql +SELECT column1 [, column2 ] +FROM table1 +[WHERE condition] + +INTERSECT + +SELECT column1 [, column2 ] +FROM table2 +[WHERE condition] + +``` +The condition could be any expression based on your requirements. + +**Example** + +Query: + +``` sql +SELECT number FROM numbers(1,10) INTERSECT SELECT number FROM numbers(3,6); +``` + +Result: + +``` text +┌─number─┐ +│ 3 │ +│ 4 │ +│ 5 │ +│ 6 │ +│ 7 │ +│ 8 │ +└────────┘ +``` + +Query: + +``` sql +SELECT number FROM numbers(1,10) WHERE number > 5 INTERSECT SELECT number FROM numbers(3,6) WHERE number < 7; +``` + +Result: + +``` text +┌─number─┐ +│ 6 │ +└────────┘ +``` + +Query: + +``` sql +CREATE TABLE t1(one String, two String, three String) ENGINE=Memory(); +CREATE TABLE t2(four String, five String, six String) ENGINE=Memory(); + +INSERT INTO t1 VALUES ('q', 'm', 'b'), ('s', 'd', 'f'), ('l', 'p', 'o'), ('s', 'd', 'f'), ('s', 'd', 'f'), ('k', 't', 'd'), ('l', 'p', 'o'); +INSERT INTO t2 VALUES ('q', 'm', 'b'), ('b', 'd', 'k'), ('s', 'y', 't'), ('s', 'd', 'f'), ('m', 'f', 'o'), ('k', 'k', 'd'); + +SELECT * FROM t1 INTERSECT SELECT * FROM t2; +``` + +Result: + +``` text +┌─one─┬─two─┬─three─┐ +│ q │ m │ b │ +│ s │ d │ f │ +│ s │ d │ f │ +│ s │ d │ f │ +└─────┴─────┴───────┘ +``` + +**See Also** + +- [UNION](union.md#union-clause) +- [EXCEPT](except.md#except-clause) \ No newline at end of file From e36f06887bcccc1bb2bfc29844415b279987da21 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Wed, 20 Oct 2021 20:43:07 +0300 Subject: [PATCH 024/173] update --- docs/en/sql-reference/operators/index.md | 2 +- docs/en/sql-reference/statements/select/intersect.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index 159fc692e94..7cc7a48cd77 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -94,7 +94,7 @@ Result: └───┘ ``` -`ANY` – The operator is comparing a value to a set of values returned by a subquery. The comparison condition must be running for any values of the subquery. +`ANY` – The operator is comparing a value to a set of values returned by a subquery. The comparison condition can be running for any values of the subquery. The subquery must select values of the same type as those compared in the main predicate. diff --git a/docs/en/sql-reference/statements/select/intersect.md b/docs/en/sql-reference/statements/select/intersect.md index 89446c68e62..95af5eb50a0 100644 --- a/docs/en/sql-reference/statements/select/intersect.md +++ b/docs/en/sql-reference/statements/select/intersect.md @@ -4,7 +4,7 @@ toc_title: INTERSECT # INTERSECT Clause {#intersect-clause} -The `INTERSECT` clause returns only those rows that result from first and second queries. The queries must match the number of columns, order, type, and names. The result of `INTERSECT` can contain duplicate rows. +The `INTERSECT` clause returns only those rows that result from first and second queries. The queries must match the number of columns, order, and type. The result of `INTERSECT` can contain duplicate rows. Multiple `INTERSECT` statements execute left to right if brackets are not specified. The `INTERSECT` operator has a higher priority than the `UNION` and `EXCEPT` clause. From 81d9bac04a35edd0338869a5c5287818f9c5bc5c Mon Sep 17 00:00:00 2001 From: gyuton Date: Fri, 22 Oct 2021 03:53:44 +0300 Subject: [PATCH 025/173] Initial draft --- docs/en/operations/settings/settings.md | 77 +++++++++++++++++++ .../sql-reference/statements/insert-into.md | 1 + 2 files changed, 78 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2b088bf45bf..c0aa94d60b2 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3814,3 +3814,80 @@ Default value: `0`. **See Also** - [optimize_move_to_prewhere](#optimize_move_to_prewhere) setting + +## async_insert {#async-insert} + +Enables or disables asynchronous inserts for multiple concurrent connetions. This makes sense only for insertion over HTTP protocol. Note, that deduplications isn't working for such inserts. + +If enabled, the data is combined into batches before the insertion into tables, so it is possible to do small and frequent insertions into ClickHouse (up to 15000 queries per second) without buffer tables. + +The data is inserted either after the [async-insert-max-data-size](#async-insert-max-data-size) is exceeded or after [async_insert_busy_timeout](#async_insert_busy_timeout) milliseconds since the first `INSERT` query. If the [async-insert-stale-timeout](#async-insert-stale-timeout) is set to a non-zero value, the data is inserted after `async-insert-stale-timeout` milliseconds since the last query. + +If [wait-for-async-insert](#wait-for-async-insert) is enabled, every client will wait for the data to be processed and flushed to the table. Otherwise, the query would be processed almost instantly, even if the data is not inserted. + +Possible values: + +- 0 — Disabled. +- 1 — Enabled. + +Default value: `0`. + +## async_insert_threads {#async-insert-threads} + +The maximum number of threads for backgroung data parsing and insertion. + +Possible values: + +- Positive integer. +- 0 — Asynchronous insertions are disabled. + +Default value: `16`. + +## wait_for_async_insert {#wait-for-async-insert} + +Enables or disables waiting for processing of asynchronous insertion. If enabled, client will return `OK` only after the data is inserted. Otherwise, it will return `OK` even if the data wasn't inserted. + +Possible values: + +- 0 — Disabled. +- 1 — Enabled. + +Default value: `1`. + +## wait_for_async_insert_timeout {#wait-for-async-insert-timeout} + +The timeout in seconds for waiting for processing of asynchronous insertion. + +- Positive integer. +- 0 — Disabled. + +Default value: [lock_acquire_timeout](#lock_acquire_timeout). + +## async_insert_max_data_size {#async-insert-max-data-size} + +The maximum size of the uparsed data in bytes collected per query before being inserted. + +Possible values: + +- Positive integer. +- 0 — Asynchronous insertions are disabled. + +Default value: `1000000`. + +## async_insert_busy_timeout {#async-insert-busy-timeout} + +The maximum timeout in milliseconds since the first `INSERT` query before inserting collected data. + +- Positive integer. +- 0 — Timeout disabled. + +Default value: `200`. + +## async_insert_stale_timeout {#async-insert-stale-timeout} + +The maximum timeout in milliseconds since the last `INSERT` query before dumping collected data. If enabled, the settings prolongs the [async_insert_busy_timeout](#async_insert_busy_timeout) with every `INSERT` query as long as [async-insert-max-data-size](#async-insert-max-data-size) is not exceeded. + +- Positive integer. +- 0 — Timeout disabled. + +Default value: `0`. diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index 31b4d30835f..1bc69d9133f 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -119,3 +119,4 @@ Performance will not decrease if: - Data is added in real time. - You upload data that is usually sorted by time. +It's also possible to asynchronously insert data in small but frequent inserts from multiple concurrent connections with the setting [async-insert](../../operations/settings/settings.md#async-insert). The data from such inserts is combined into batches and then safely inserted into a table. From 8f039a2b0f35611e2ab8c1e3e5ebe9577e4fc550 Mon Sep 17 00:00:00 2001 From: gyuton Date: Fri, 22 Oct 2021 04:04:15 +0300 Subject: [PATCH 026/173] fixed links --- docs/en/operations/settings/settings.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index c0aa94d60b2..f02fe383f37 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3821,9 +3821,9 @@ Enables or disables asynchronous inserts for multiple concurrent connetions. Thi If enabled, the data is combined into batches before the insertion into tables, so it is possible to do small and frequent insertions into ClickHouse (up to 15000 queries per second) without buffer tables. -The data is inserted either after the [async-insert-max-data-size](#async-insert-max-data-size) is exceeded or after [async_insert_busy_timeout](#async_insert_busy_timeout) milliseconds since the first `INSERT` query. If the [async-insert-stale-timeout](#async-insert-stale-timeout) is set to a non-zero value, the data is inserted after `async-insert-stale-timeout` milliseconds since the last query. +The data is inserted either after the [async_insert_max_data_size](#async-insert-max-data-size) is exceeded or after [async_insert_busy_timeout](#async-insert-busy-timeout) milliseconds since the first `INSERT` query. If the [async_insert_stale_timeout](#async-insert-stale-timeout) is set to a non-zero value, the data is inserted after `async_insert_stale_timeout` milliseconds since the last query. -If [wait-for-async-insert](#wait-for-async-insert) is enabled, every client will wait for the data to be processed and flushed to the table. Otherwise, the query would be processed almost instantly, even if the data is not inserted. +If [wait_for_async_insert](#wait-for-async-insert) is enabled, every client will wait for the data to be processed and flushed to the table. Otherwise, the query would be processed almost instantly, even if the data is not inserted. Possible values: @@ -3885,7 +3885,7 @@ Default value: `200`. ## async_insert_stale_timeout {#async-insert-stale-timeout} -The maximum timeout in milliseconds since the last `INSERT` query before dumping collected data. If enabled, the settings prolongs the [async_insert_busy_timeout](#async_insert_busy_timeout) with every `INSERT` query as long as [async-insert-max-data-size](#async-insert-max-data-size) is not exceeded. +The maximum timeout in milliseconds since the last `INSERT` query before dumping collected data. If enabled, the settings prolongs the [async_insert_busy_timeout](#async-insert-busy-timeout) with every `INSERT` query as long as [async_insert_max_data_size](#async-insert-max-data-size) is not exceeded. - Positive integer. - 0 — Timeout disabled. From 4ecd2ed6e204d4d70200810c43d5370dcbd61ee9 Mon Sep 17 00:00:00 2001 From: gyuton Date: Fri, 22 Oct 2021 04:21:29 +0300 Subject: [PATCH 027/173] typo --- docs/en/sql-reference/statements/insert-into.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index 1bc69d9133f..83812a198d9 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -119,4 +119,4 @@ Performance will not decrease if: - Data is added in real time. - You upload data that is usually sorted by time. -It's also possible to asynchronously insert data in small but frequent inserts from multiple concurrent connections with the setting [async-insert](../../operations/settings/settings.md#async-insert). The data from such inserts is combined into batches and then safely inserted into a table. +It's also possible to asynchronously insert data in small but frequent inserts from multiple concurrent connections with the setting [async_insert](../../operations/settings/settings.md#async-insert). The data from such inserts is combined into batches and then safely inserted into a table. From 1e203cb604c956b5dcdbfbe01226930f880c8ee9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Fri, 22 Oct 2021 14:58:10 +0800 Subject: [PATCH 028/173] Retranslation --- .../system-tables/asynchronous_metric_log.md | 32 ++++++++++++++++++- 1 file changed, 31 insertions(+), 1 deletion(-) diff --git a/docs/zh/operations/system-tables/asynchronous_metric_log.md b/docs/zh/operations/system-tables/asynchronous_metric_log.md index ff7593768d3..6ae4a600539 100644 --- a/docs/zh/operations/system-tables/asynchronous_metric_log.md +++ b/docs/zh/operations/system-tables/asynchronous_metric_log.md @@ -5,4 +5,34 @@ machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ## system.asynchronous_metric_log {#system-tables-async-log} -包含以下内容的历史值 `system.asynchronous_log` (见 [系统。asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics)) +包含每分钟记录一次的 `system.asynchronous_metrics`历史值. 默认开启. + +列: +- `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)) — 指标值. + +**示例** +``` sql +SELECT * FROM system.asynchronous_metric_log LIMIT 10 +``` +``` text +┌─event_date─┬──────────event_time─┬────event_time_microseconds─┬─name─────────────────────────────────────┬─────value─┐ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ CPUFrequencyMHz_0 │ 2120.9 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.arenas.all.pmuzzy │ 743 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.arenas.all.pdirty │ 26288 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.background_thread.run_intervals │ 0 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.background_thread.num_runs │ 0 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.retained │ 60694528 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.mapped │ 303161344 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.resident │ 260931584 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.metadata │ 12079488 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.allocated │ 133756128 │ +└────────────┴─────────────────────┴────────────────────────────┴──────────────────────────────────────────┴───────────┘ +``` + +**另请参阅** +- [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — 包含在后台定期计算的指标. +- [system.metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — 包含定期刷新到磁盘表 `system.metrics` 以及 `system.events` 表中指标值的隶属记录. From bb8401546a31c3b19f792f783b0b109e31384b3e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Fri, 22 Oct 2021 15:04:17 +0800 Subject: [PATCH 029/173] retranslation --- .../system-tables/asynchronous_metric_log.md | 32 ++++++++++++++++++- 1 file changed, 31 insertions(+), 1 deletion(-) diff --git a/docs/zh/operations/system-tables/asynchronous_metric_log.md b/docs/zh/operations/system-tables/asynchronous_metric_log.md index ff7593768d3..6ae4a600539 100644 --- a/docs/zh/operations/system-tables/asynchronous_metric_log.md +++ b/docs/zh/operations/system-tables/asynchronous_metric_log.md @@ -5,4 +5,34 @@ machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ## system.asynchronous_metric_log {#system-tables-async-log} -包含以下内容的历史值 `system.asynchronous_log` (见 [系统。asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics)) +包含每分钟记录一次的 `system.asynchronous_metrics`历史值. 默认开启. + +列: +- `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)) — 指标值. + +**示例** +``` sql +SELECT * FROM system.asynchronous_metric_log LIMIT 10 +``` +``` text +┌─event_date─┬──────────event_time─┬────event_time_microseconds─┬─name─────────────────────────────────────┬─────value─┐ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ CPUFrequencyMHz_0 │ 2120.9 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.arenas.all.pmuzzy │ 743 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.arenas.all.pdirty │ 26288 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.background_thread.run_intervals │ 0 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.background_thread.num_runs │ 0 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.retained │ 60694528 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.mapped │ 303161344 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.resident │ 260931584 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.metadata │ 12079488 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.allocated │ 133756128 │ +└────────────┴─────────────────────┴────────────────────────────┴──────────────────────────────────────────┴───────────┘ +``` + +**另请参阅** +- [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — 包含在后台定期计算的指标. +- [system.metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — 包含定期刷新到磁盘表 `system.metrics` 以及 `system.events` 表中指标值的隶属记录. From 4ce26e173889ad1b1be0df43d1407f64dfd3ad98 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Fri, 22 Oct 2021 15:59:17 +0800 Subject: [PATCH 030/173] Correct incorrect text --- docs/zh/operations/system-tables/asynchronous_metric_log.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/zh/operations/system-tables/asynchronous_metric_log.md b/docs/zh/operations/system-tables/asynchronous_metric_log.md index 6ae4a600539..60a3fd868a0 100644 --- a/docs/zh/operations/system-tables/asynchronous_metric_log.md +++ b/docs/zh/operations/system-tables/asynchronous_metric_log.md @@ -5,7 +5,7 @@ machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ## system.asynchronous_metric_log {#system-tables-async-log} -包含每分钟记录一次的 `system.asynchronous_metrics`历史值. 默认开启. +包含每分钟记录一次的 `system.asynchronous_metrics`表历史值. 默认开启. 列: - `event_date` ([Date](../../sql-reference/data-types/date.md)) — 事件日期. @@ -35,4 +35,4 @@ SELECT * FROM system.asynchronous_metric_log LIMIT 10 **另请参阅** - [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — 包含在后台定期计算的指标. -- [system.metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — 包含定期刷新到磁盘表 `system.metrics` 以及 `system.events` 表中指标值的隶属记录. +- [system.metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — 包含定期刷新到磁盘表 `system.metrics` 以及 `system.events` 中的指标值历史记录. From 44136e3da977c6e73348f84196e2f93a97ff8f86 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 11:46:44 +0300 Subject: [PATCH 031/173] Update docs/en/sql-reference/operators/index.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/operators/index.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index 7cc7a48cd77..134d67b0e37 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -71,7 +71,10 @@ ClickHouse transforms operators to their corresponding functions at the query pa `a GLOBAL NOT IN ...` – The `globalNotIn(a, b)` function. -`ALL` – The operator allows you to get data by comparing the value with the list of values returned by the subquery. The comparison condition must be running for all values of the subquery. +`a = ANY (subquery)` – The `in(a, subquery)` function. +`a != ANY (subquery)` – The `notIn(a, subquery)` function. +`a = ALL (subquery)` – The same as `a IN (SELECT singleValueOrNull(*) FROM subquery)`. +`a != ALL (subquery)` – The same as `a NOT IN (subquery)`. The subquery must select values of the same type as those compared in the main predicate. From 0104af6c86d622b0afff604704bd66fa3d78adf3 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 11:46:56 +0300 Subject: [PATCH 032/173] Update docs/en/sql-reference/operators/index.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/operators/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index 134d67b0e37..a2d66481ddd 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -78,7 +78,7 @@ ClickHouse transforms operators to their corresponding functions at the query pa The subquery must select values of the same type as those compared in the main predicate. -**Example** +**Examples** Query: From 71d41421a08a3a517cfdc10a0f8af4c4edae5f3f Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 11:47:11 +0300 Subject: [PATCH 033/173] Update docs/en/sql-reference/operators/index.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/operators/index.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index a2d66481ddd..a50c245de1b 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -76,7 +76,6 @@ ClickHouse transforms operators to their corresponding functions at the query pa `a = ALL (subquery)` – The same as `a IN (SELECT singleValueOrNull(*) FROM subquery)`. `a != ALL (subquery)` – The same as `a NOT IN (subquery)`. -The subquery must select values of the same type as those compared in the main predicate. **Examples** From 95cb825d214154dbf65ecb17a0d72cf5f8a3306c Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 11:47:22 +0300 Subject: [PATCH 034/173] Update docs/en/sql-reference/operators/index.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/operators/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index a50c245de1b..3583d172939 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -79,7 +79,7 @@ ClickHouse transforms operators to their corresponding functions at the query pa **Examples** -Query: +Query with ALL: ``` sql SELECT number AS a FROM numbers(10) WHERE a > all(SELECT number FROM numbers(3, 3)); From 29618ff47d42a707975a303da6c321962f721b62 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 11:47:29 +0300 Subject: [PATCH 035/173] Update docs/en/sql-reference/operators/index.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/operators/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index 3583d172939..3370c2ee21e 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -82,7 +82,7 @@ ClickHouse transforms operators to their corresponding functions at the query pa Query with ALL: ``` sql -SELECT number AS a FROM numbers(10) WHERE a > all(SELECT number FROM numbers(3, 3)); +SELECT number AS a FROM numbers(10) WHERE a > ALL (SELECT number FROM numbers(3, 3)); ``` Result: From c7cc5a99f52a21f871269242cd686245b6ab0186 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 11:47:41 +0300 Subject: [PATCH 036/173] Update docs/en/sql-reference/operators/index.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/operators/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index 3370c2ee21e..4e3b539c731 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -102,7 +102,7 @@ The subquery must select values of the same type as those compared in the main p **Example** -Query: +Query with ANY: ``` sql SELECT number AS a FROM numbers(10) WHERE a > any(SELECT number FROM numbers(3, 3)); From bb7a179f9314f302e56ac9c58dfb92767e5c7072 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 11:47:48 +0300 Subject: [PATCH 037/173] Update docs/en/sql-reference/operators/index.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/operators/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index 4e3b539c731..f4aadababdb 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -105,7 +105,7 @@ The subquery must select values of the same type as those compared in the main p Query with ANY: ``` sql -SELECT number AS a FROM numbers(10) WHERE a > any(SELECT number FROM numbers(3, 3)); +SELECT number AS a FROM numbers(10) WHERE a > ANY (SELECT number FROM numbers(3, 3)); ``` Result: From cb907dd6c60d7e8f2d94d49792e6ae55c4c55b03 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 11:48:07 +0300 Subject: [PATCH 038/173] Update docs/en/sql-reference/statements/select/intersect.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/select/intersect.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/intersect.md b/docs/en/sql-reference/statements/select/intersect.md index 95af5eb50a0..58f8893a7a0 100644 --- a/docs/en/sql-reference/statements/select/intersect.md +++ b/docs/en/sql-reference/statements/select/intersect.md @@ -4,7 +4,7 @@ toc_title: INTERSECT # INTERSECT Clause {#intersect-clause} -The `INTERSECT` clause returns only those rows that result from first and second queries. The queries must match the number of columns, order, and type. The result of `INTERSECT` can contain duplicate rows. +The `INTERSECT` clause returns only those rows that result from both the first and the second queries. The queries must match the number of columns, order, and type. The result of `INTERSECT` can contain duplicate rows. Multiple `INTERSECT` statements execute left to right if brackets are not specified. The `INTERSECT` operator has a higher priority than the `UNION` and `EXCEPT` clause. From 1b408ed588ff7ffa3e4aea8b5a311562af785544 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 11:48:31 +0300 Subject: [PATCH 039/173] Update docs/en/sql-reference/operators/index.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/operators/index.md | 6 ------ 1 file changed, 6 deletions(-) diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index f4aadababdb..4ee22025674 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -96,12 +96,6 @@ Result: └───┘ ``` -`ANY` – The operator is comparing a value to a set of values returned by a subquery. The comparison condition can be running for any values of the subquery. - -The subquery must select values of the same type as those compared in the main predicate. - -**Example** - Query with ANY: ``` sql From 41f69e967712bb250e48dac2494888b03b01db54 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 11:48:53 +0300 Subject: [PATCH 040/173] Update docs/en/sql-reference/statements/select/except.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/select/except.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/except.md b/docs/en/sql-reference/statements/select/except.md index ffbc0fae3da..6702043e6b0 100644 --- a/docs/en/sql-reference/statements/select/except.md +++ b/docs/en/sql-reference/statements/select/except.md @@ -6,7 +6,7 @@ toc_title: EXCEPT The `EXCEPT` clause returns only those rows that result from the first query without the second. The queries must match the number of columns, order, and type. The result of `EXCEPT` can contain duplicate rows. -Multiple `EXCEPT` statements execute left to right if brackets are not specified. The `EXCEPT` operator has the same priority as the `UNION` clause and lower priority as the `INTERSECT` clause. +Multiple `EXCEPT` statements are executed left to right if parenthesis are not specified. The `EXCEPT` operator has the same priority as the `UNION` clause and lower priority than the `INTERSECT` clause. ``` sql SELECT column1 [, column2 ] From 692b99ca0d9e5e48cc7156fd68e12cf7576f421f Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 11:49:01 +0300 Subject: [PATCH 041/173] Update docs/en/sql-reference/statements/select/intersect.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/select/intersect.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/intersect.md b/docs/en/sql-reference/statements/select/intersect.md index 58f8893a7a0..cc75a2e2b4d 100644 --- a/docs/en/sql-reference/statements/select/intersect.md +++ b/docs/en/sql-reference/statements/select/intersect.md @@ -6,7 +6,7 @@ toc_title: INTERSECT The `INTERSECT` clause returns only those rows that result from both the first and the second queries. The queries must match the number of columns, order, and type. The result of `INTERSECT` can contain duplicate rows. -Multiple `INTERSECT` statements execute left to right if brackets are not specified. The `INTERSECT` operator has a higher priority than the `UNION` and `EXCEPT` clause. +Multiple `INTERSECT` statements are executes left to right if parenthesis are not specified. The `INTERSECT` operator has a higher priority than the `UNION` and `EXCEPT` clause. ``` sql From 0f66ba1704a21423fbf5d5b60ed7557cd3ea1c2d Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 11:49:11 +0300 Subject: [PATCH 042/173] Update docs/en/sql-reference/statements/select/intersect.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/select/intersect.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/intersect.md b/docs/en/sql-reference/statements/select/intersect.md index cc75a2e2b4d..4a222d80d8d 100644 --- a/docs/en/sql-reference/statements/select/intersect.md +++ b/docs/en/sql-reference/statements/select/intersect.md @@ -23,7 +23,7 @@ FROM table2 ``` The condition could be any expression based on your requirements. -**Example** +**Examples** Query: From 745b47762cdd24cf6ec762ce28ced0e245514c60 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 11:49:20 +0300 Subject: [PATCH 043/173] Update docs/en/sql-reference/statements/select/except.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/select/except.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/except.md b/docs/en/sql-reference/statements/select/except.md index 6702043e6b0..362ddb29b0e 100644 --- a/docs/en/sql-reference/statements/select/except.md +++ b/docs/en/sql-reference/statements/select/except.md @@ -22,7 +22,7 @@ FROM table2 ``` The condition could be any expression based on your requirements. -**Example** +**Examples** Query: From ca6b6df93e7a7bc71d8f275ed935415ba0deaad5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Fri, 22 Oct 2021 16:49:24 +0800 Subject: [PATCH 044/173] translation for Continuous Integration Checks Document: Chinese translation for Continuous Integration Checks --- docs/zh/development/continuous-integration.md | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/docs/zh/development/continuous-integration.md b/docs/zh/development/continuous-integration.md index f68058a436e..8f147b68cf1 120000 --- a/docs/zh/development/continuous-integration.md +++ b/docs/zh/development/continuous-integration.md @@ -1 +1,18 @@ -../../en/development/continuous-integration.md \ No newline at end of file +# 持续集成检查 {#continuous-integration-checks} +当你提交一个pull请求时, ClickHouse的[持续集成(CI)系统](../tests.md#test-automation)会对您的代码运行一些自动检查. + +这在存储库维护者(来自ClickHouse团队的人)筛选了您的代码并将可测试标签添加到您的pull请求之后发生. + +检查的结果被列在[GitHub检查文档](https://docs.github.com/en/github/collaborating-with-pull-requests/collaborating-on-repositories-with-code-quality-features/about-status-checks)中描述的GitHub pull请求页面. + +如果检查失败,您可能被要求去修复它. 该界面介绍了您可能遇到的检查,以及如何修复它们. + +如果检查失败看起来与您的更改无关,那么它可能是一些暂时的故障或基础设施问题. 向pull请求推一个空的commit以重新启动CI检查: + +``` +git reset +git commit --allow-empty +git push +``` + +如果您不确定要做什么,可以向维护人员寻求帮助. From 3510f4cddcc8c07f4358cb5dc3c1c8829a95090e Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 11:58:04 +0300 Subject: [PATCH 045/173] Update except.md --- .../sql-reference/statements/select/except.md | 19 +------------------ 1 file changed, 1 insertion(+), 18 deletions(-) diff --git a/docs/en/sql-reference/statements/select/except.md b/docs/en/sql-reference/statements/select/except.md index 362ddb29b0e..d45c0556a1a 100644 --- a/docs/en/sql-reference/statements/select/except.md +++ b/docs/en/sql-reference/statements/select/except.md @@ -43,23 +43,6 @@ Result: Query: -``` sql -SELECT number FROM numbers(1,10) WHERE number > 5 EXCEPT SELECT number FROM numbers(3,6) WHERE number < 7; -``` - -Result: - -``` text -┌─number─┐ -│ 7 │ -│ 8 │ -│ 9 │ -│ 10 │ -└────────┘ -``` - -Query: - ``` sql CREATE TABLE t1(one String, two String, three String) ENGINE=Memory(); CREATE TABLE t2(four String, five String, six String) ENGINE=Memory(); @@ -83,4 +66,4 @@ Result: **See Also** - [UNION](union.md#union-clause) -- [INTERSECT](except.md#except-clause) \ No newline at end of file +- [INTERSECT](except.md#except-clause) From e4df8417981ab0d6e106454965685cc89341c657 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 11:58:34 +0300 Subject: [PATCH 046/173] Update intersect.md --- .../sql-reference/statements/select/intersect.md | 16 +--------------- 1 file changed, 1 insertion(+), 15 deletions(-) diff --git a/docs/en/sql-reference/statements/select/intersect.md b/docs/en/sql-reference/statements/select/intersect.md index 4a222d80d8d..2243a35e4d8 100644 --- a/docs/en/sql-reference/statements/select/intersect.md +++ b/docs/en/sql-reference/statements/select/intersect.md @@ -46,20 +46,6 @@ Result: Query: -``` sql -SELECT number FROM numbers(1,10) WHERE number > 5 INTERSECT SELECT number FROM numbers(3,6) WHERE number < 7; -``` - -Result: - -``` text -┌─number─┐ -│ 6 │ -└────────┘ -``` - -Query: - ``` sql CREATE TABLE t1(one String, two String, three String) ENGINE=Memory(); CREATE TABLE t2(four String, five String, six String) ENGINE=Memory(); @@ -84,4 +70,4 @@ Result: **See Also** - [UNION](union.md#union-clause) -- [EXCEPT](except.md#except-clause) \ No newline at end of file +- [EXCEPT](except.md#except-clause) From fc3f577cbfba6809d51f8900ec1a14866c7fc00f Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 12:14:33 +0300 Subject: [PATCH 047/173] Update index.md --- docs/en/sql-reference/operators/index.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index 4ee22025674..d23c81e2aca 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -72,8 +72,11 @@ ClickHouse transforms operators to their corresponding functions at the query pa `a GLOBAL NOT IN ...` – The `globalNotIn(a, b)` function. `a = ANY (subquery)` – The `in(a, subquery)` function. + `a != ANY (subquery)` – The `notIn(a, subquery)` function. + `a = ALL (subquery)` – The same as `a IN (SELECT singleValueOrNull(*) FROM subquery)`. + `a != ALL (subquery)` – The same as `a NOT IN (subquery)`. From 3c8b32dfe6f46973ee533a18db99d240f31e9e29 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Fri, 22 Oct 2021 17:20:10 +0800 Subject: [PATCH 048/173] =?UTF-8?q?=E7=AC=AC=E4=BA=8C=E6=AE=B5=E7=BF=BB?= =?UTF-8?q?=E8=AF=91=E5=AD=98=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/zh/development/continuous-integration.md | 35 ++++++++++++++++++- 1 file changed, 34 insertions(+), 1 deletion(-) diff --git a/docs/zh/development/continuous-integration.md b/docs/zh/development/continuous-integration.md index 8f147b68cf1..70a88702a92 120000 --- a/docs/zh/development/continuous-integration.md +++ b/docs/zh/development/continuous-integration.md @@ -7,7 +7,7 @@ 如果检查失败,您可能被要求去修复它. 该界面介绍了您可能遇到的检查,以及如何修复它们. -如果检查失败看起来与您的更改无关,那么它可能是一些暂时的故障或基础设施问题. 向pull请求推一个空的commit以重新启动CI检查: +如果检查失败看起来与您的更改无关, 那么它可能是一些暂时的故障或基础设施问题. 向pull请求推一个空的commit以重新启动CI检查: ``` git reset @@ -16,3 +16,36 @@ git push ``` 如果您不确定要做什么,可以向维护人员寻求帮助. + + +# 与Master合并 {#merge-with-master} +验证PR是否可以合并到master. 如果没有, 它将返回消息为'Cannot fetch mergecommit'的失败.要修复这个检查, 解决[GitHub文档](https://docs.github.com/en/github/collaborating-with-pull-requests/addressing-merge-conflicts/resolving-a-merge-conflict-on-github)中描述的冲突, 或者使用git合并主分支到你的pull request分支. + +# 文档检查 {#docs-check} +尝试构建ClickHouse文档网站. 如果您更改了文档中的某些内容, 它可能会失败. 最可能的原因是文档中的某些交叉链接是错误的. 转到检查报告并查找`ERROR`和`WARNING`消息. + +# 查看报告详情 {#report-details} +- [状态页案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) +- `docs_output.txt`记录了build日志信息. [成功结果案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check/docs_output.txt) + +# 描述信息检查 {#description-check} +检查pull请求的描述是否符合[PULL_REQUEST_TEMPLATE.md](https://github.com/ClickHouse/ClickHouse/blob/master/.github/PULL_REQUEST_TEMPLATE.md)模板. + +您必须为您的更改指定一个更改日志类别(例如,Bug修复), 并且为[CHANGELOG.md](https://clickhouse.com/docs/en/whats-new/changelog/)编写一条用户可读的消息用来描述更改. + +# 推送到DockerHub {#push-to-dockerhub} +生成用于构建和测试的docker映像, 然后将它们推送到DockerHub. + +# 标记检查 {#marker-check} +该检查意味着集成系统已经开始处理PR.当它处于待定状态时, 意味着还没有开始所有的检查. 在所有检查启动后,它将状态更改为'成功'. + +# 格式检查 {#style-check} +使用`utils/check-style/check-style`二进制文件执行一些简单的基于正则表达式的代码样式检查(注意, 它可以在本地运行). +如果失败, 按照[代码样式指南](https://clickhouse.com/docs/en/development/style/)修复样式错误. + +# 查看报告详情 {#report-details} +- [状态页案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) +- `docs_output.txt`记录了查结果错误(无效表格等), 空白页表示没有错误. [成功结果案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check/docs_output.txt) + +# PVS Check {#pvs-check} +使用静态分析工具[PVS-studio](https://www.viva64.com/en/pvs-studio/)检查代码. 查看报告找出确切的错误.如果可以则修复它们, 如果不行, 可以请ClickHouse的维护人员帮忙. From 40e35a4571b8617dda035e27f232459aaec00779 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Fri, 22 Oct 2021 17:58:02 +0800 Subject: [PATCH 049/173] Archive translated documents --- docs/zh/development/continuous-integration.md | 65 +++++++++++++++++++ 1 file changed, 65 insertions(+) diff --git a/docs/zh/development/continuous-integration.md b/docs/zh/development/continuous-integration.md index 70a88702a92..205a7edf15e 120000 --- a/docs/zh/development/continuous-integration.md +++ b/docs/zh/development/continuous-integration.md @@ -49,3 +49,68 @@ git push # PVS Check {#pvs-check} 使用静态分析工具[PVS-studio](https://www.viva64.com/en/pvs-studio/)检查代码. 查看报告找出确切的错误.如果可以则修复它们, 如果不行, 可以请ClickHouse的维护人员帮忙. + +# 查看报告详情 {#report-details} +- [状态页案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) +- `test_run.txt.out.log`记录了构建和分析日志文件.它只包含解析或未找到的错误. +- `HTML report`记录了分析结果.关于它的描述, 请访问PVS的[官方网站](https://www.viva64.com/en/m/0036/#ID14E9A2B2CD) + +# {#fast-test} +通常情况下这是PR运行的第一个检查.它构建ClickHouse以及大多数无状态运行测试, 其中省略了一些.如果失败,在修复之前不会开始进一步的检查. 通过报告查看哪些测试失败, 然后按照[这里](https://clickhouse.com/docs/en/development/tests/#functional-test-locally)描述的在本地重现失败. + +# 查看报告详情 {#report-details} +[状态页案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) + +# 状态页文件 {#status-page-files} +- `runlog.out.log` 是包含所有其他日志的通用日志. +- `test_log.txt` +- `submodule_log.txt` 包含关于克隆和检查所需子模块的消息. +- `stderr.log` +- `stdout.log` +- `clickhouse-server.log` +- `clone_log.txt` +- `install_log.txt` +- `clickhouse-server.err.log` +- `build_log.txt` +- `cmake_log.txt` 包含关于C/ c++和Linux标志检查的消息. + +# 状态页列信息 {#status-page-columns} +- 测试名称 -- 包含测试的名称(没有路径, 例如, 所有类型的测试将被剥离到该名称). +- 测试状态 -- 跳过、成功或失败. +- 测试时间, 等等. -- 这个测试是空的. + +# 建构检查 {#build-check} +在各种配置中构建ClickHouse, 以便在后续步骤中使用. 您必须修复失败的构建.构建日志通常有足够的信息来修复错误, 但是您可能必须在本地重新生成错误. cmake选项可以在构建日志中找到, 为cmake做准备.使用这些选项并遵循一般的构建过程. + +# 查看报告详情 {#report-details} +[状态页案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) +- **Compiler**: `gcc-9` or `clang-10` (or `clang-10-xx` for other architectures e.g. `clang-10-freebsd`). +- **Build type**: `Debug` or `RelWithDebInfo` (cmake). +- **Sanitizer**: `none` (without sanitizers), `address` (ASan), `memory` (MSan), `undefined` (UBSan), or `thread` (TSan). +- **Bundled**: `bundled` 构建使用来自 `contrib` 库, 而 `unbundled` 构建使用系统库. +- **Splitted**: `splitted` is a [split build](https://clickhouse.com/docs/en/development/build/#split-build) +- **Status**: `成功` 或 `失败` +- **Build log**: 链接到生成和文件复制日志,这在生成失败时很有用. +- **Build time**. +- **Artifacts**: 构建结果文件 (with `XXX` being the server version e.g. `20.8.1.4344`). + - `clickhouse-client_XXX_all.deb` + -` clickhouse-common-static-dbg_XXX[+asan, +msan, +ubsan, +tsan]_amd64.deb` + - `clickhouse-common-staticXXX_amd64.deb` + - `clickhouse-server_XXX_all.deb` + - `clickhouse-test_XXX_all.deb` + - `clickhouse_XXX_amd64.buildinfo` + - `clickhouse_XXX_amd64.changes` + - `clickhouse`: Main built binary. + - `clickhouse-odbc-bridge` + - `unit_tests_dbms`: GoogleTest二进制和ClickHouse单元测试. + - `shared_build.tgz`: 使用共享库构建. + - `performance.tgz`: 用于性能测试的特殊包. + +# 特殊的构建检查 {#special-buildcheck} +使用clang-tidy执行静态分析和代码风格检查. 报告类似于构建检查. 修复在构建日志中发现的错误. + + + + + + From add7c000e19c3c95df19556fd724aaa2d7dee084 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 22 Oct 2021 16:32:49 +0300 Subject: [PATCH 050/173] Update test_aggregate_fixed_key.py --- .../test_backward_compatibility/test_aggregate_fixed_key.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py index d895ee0eb78..8819be527fd 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py @@ -3,9 +3,9 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', with_zookeeper=True, image='clickhouse/server', tag='21.3', with_installed_binary=True) -node2 = cluster.add_instance('node2', with_zookeeper=True, image='clickhouse/server') -node3 = cluster.add_instance('node3', with_zookeeper=True, image='clickhouse/server') +node1 = cluster.add_instance('node1', with_zookeeper=True, image='yandex/clickhouse-server', tag='21.3', with_installed_binary=True) +node2 = cluster.add_instance('node2', with_zookeeper=True, image='yandex/clickhouse-server') +node3 = cluster.add_instance('node3', with_zookeeper=True, image='yandex/clickhouse-server') @pytest.fixture(scope="module") From e00d55c08d416300810ccc52a77bbf95c4caab67 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 22 Oct 2021 17:42:20 +0300 Subject: [PATCH 051/173] Try to fix 30397 --- src/Storages/StorageBuffer.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index a74223054e9..1d1c3456813 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -817,8 +817,6 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds, bool loc if (!locked) lock.emplace(buffer.lockForReading()); - block_to_write = buffer.data.cloneEmpty(); - rows = buffer.data.rows(); bytes = buffer.data.bytes(); if (buffer.first_write_time) From ee6bc3883fd2df7801be1f5de23a07cb2d889365 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 22 Oct 2021 19:25:48 +0300 Subject: [PATCH 052/173] Try to fix 30397 --- src/Storages/StorageBuffer.cpp | 22 ++++++++++++++++------ 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 1d1c3456813..549f87c9f99 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -446,7 +446,8 @@ static void appendBlock(const Block & from, Block & to) if (!to) throw Exception("Cannot append to empty block", ErrorCodes::LOGICAL_ERROR); - assertBlocksHaveEqualStructure(from, to, "Buffer"); + if (to.rows()) + assertBlocksHaveEqualStructure(from, to, "Buffer"); from.checkNumberOfRows(); to.checkNumberOfRows(); @@ -464,14 +465,21 @@ static void appendBlock(const Block & from, Block & to) { MemoryTracker::BlockerInThread temporarily_disable_memory_tracker; - for (size_t column_no = 0, columns = to.columns(); column_no < columns; ++column_no) + if (to.rows() == 0) { - const IColumn & col_from = *from.getByPosition(column_no).column.get(); - last_col = IColumn::mutate(std::move(to.getByPosition(column_no).column)); + to = from; + } + else + { + for (size_t column_no = 0, columns = to.columns(); column_no < columns; ++column_no) + { + const IColumn & col_from = *from.getByPosition(column_no).column.get(); + last_col = IColumn::mutate(std::move(to.getByPosition(column_no).column)); - last_col->insertRangeFrom(col_from, 0, rows); + last_col->insertRangeFrom(col_from, 0, rows); - to.getByPosition(column_no).column = std::move(last_col); + to.getByPosition(column_no).column = std::move(last_col); + } } } catch (...) @@ -817,6 +825,8 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds, bool loc if (!locked) lock.emplace(buffer.lockForReading()); + block_to_write = buffer.data.cloneEmpty(); + rows = buffer.data.rows(); bytes = buffer.data.bytes(); if (buffer.first_write_time) From ae692b96c6ac2d1c296f7507fc6a9b778dd458d7 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 21:57:38 +0300 Subject: [PATCH 053/173] russian --- docs/ru/sql-reference/operators/index.md | 47 ++++++++++++ .../sql-reference/statements/select/except.md | 69 ++++++++++++++++++ .../sql-reference/statements/select/index.md | 4 +- .../statements/select/intersect.md | 73 +++++++++++++++++++ 4 files changed, 192 insertions(+), 1 deletion(-) create mode 100644 docs/ru/sql-reference/statements/select/except.md create mode 100644 docs/ru/sql-reference/statements/select/intersect.md diff --git a/docs/ru/sql-reference/operators/index.md b/docs/ru/sql-reference/operators/index.md index 98f6873f712..19119f641ef 100644 --- a/docs/ru/sql-reference/operators/index.md +++ b/docs/ru/sql-reference/operators/index.md @@ -72,6 +72,53 @@ toc_title: "Операторы" `a GLOBAL NOT IN ...` - функция `globalNotIn(a, b)` +`a = ANY (subquery)` – функция `in(a, subquery)` + +`a != ANY (subquery)` – функция `notIn(a, subquery)` + +`a = ALL (subquery)` – равнозначно `a IN (SELECT singleValueOrNull(*) FROM subquery)` + +`a != ALL (subquery)` – равнозначно `a NOT IN (subquery)` + + +**Примеры** + +Запрос с ALL: + +``` sql +SELECT number AS a FROM numbers(10) WHERE a > ALL (SELECT number FROM numbers(3, 3)); +``` + +Результат: + +``` text +┌─a─┐ +│ 6 │ +│ 7 │ +│ 8 │ +│ 9 │ +└───┘ +``` + +Запрос с ANY: + +``` sql +SELECT number AS a FROM numbers(10) WHERE a > ANY (SELECT number FROM numbers(3, 3)); +``` + +Результат: + +``` text +┌─a─┐ +│ 4 │ +│ 5 │ +│ 6 │ +│ 7 │ +│ 8 │ +│ 9 │ +└───┘ +``` + ## Оператор для работы с датами и временем {#operators-datetime} ### EXTRACT {#extract} diff --git a/docs/ru/sql-reference/statements/select/except.md b/docs/ru/sql-reference/statements/select/except.md new file mode 100644 index 00000000000..6929eff1c81 --- /dev/null +++ b/docs/ru/sql-reference/statements/select/except.md @@ -0,0 +1,69 @@ +--- +toc_title: EXCEPT +--- + +# Секция EXCEPT {#except-clause} + +Оператор `EXCEPT` возвращает только те строки, которые являются результатом первого запроса без результатов второго. В запросах количество, порядок следования и типы столбцов должны совпадать. Результат `EXCEPT` может содержать повторяющиеся строки. + +Если используется несколько `EXCEPT`, и в выражении не указаны скобки, `EXCEPT` выполняется по порядку слева направо. Оператор `EXCEPT` имеет такой же приоритет выполнения, как `UNION`, и ниже, чем `INTERSECT`. + +``` sql +SELECT column1 [, column2 ] +FROM table1 +[WHERE condition] + +EXCEPT + +SELECT column1 [, column2 ] +FROM table2 +[WHERE condition] + +``` +Условие в секции `WHERE` может быть любым в зависимости от ваших требований. + +**Примеры** + +Запрос: + +``` sql +SELECT number FROM numbers(1,10) EXCEPT SELECT number FROM numbers(3,6); +``` + +Результат: + +``` text +┌─number─┐ +│ 1 │ +│ 2 │ +│ 9 │ +│ 10 │ +└────────┘ +``` + +Запрос: + +``` sql +CREATE TABLE t1(one String, two String, three String) ENGINE=Memory(); +CREATE TABLE t2(four String, five String, six String) ENGINE=Memory(); + +INSERT INTO t1 VALUES ('q', 'm', 'b'), ('s', 'd', 'f'), ('l', 'p', 'o'), ('s', 'd', 'f'), ('s', 'd', 'f'), ('k', 't', 'd'), ('l', 'p', 'o'); +INSERT INTO t2 VALUES ('q', 'm', 'b'), ('b', 'd', 'k'), ('s', 'y', 't'), ('s', 'd', 'f'), ('m', 'f', 'o'), ('k', 'k', 'd'); + +SELECT * FROM t1 EXCEPT SELECT * FROM t2; +``` + +Результат: + +``` text +┌─one─┬─two─┬─three─┐ +│ l │ p │ o │ +│ k │ t │ d │ +│ l │ p │ o │ +└─────┴─────┴───────┘ +``` + +**См. также** + +- [UNION](union.md#union-clause) +- [INTERSECT](except.md#except-clause) \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/select/index.md b/docs/ru/sql-reference/statements/select/index.md index ffaae74b1d9..42490a6c2f7 100644 --- a/docs/ru/sql-reference/statements/select/index.md +++ b/docs/ru/sql-reference/statements/select/index.md @@ -45,8 +45,10 @@ SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list - [Секция LIMIT BY](limit-by.md) - [Секция HAVING](having.md) - [Секция LIMIT](limit.md) - [Секция OFFSET](offset.md) +- [Секция OFFSET](offset.md) - [Секция UNION ALL](union.md) +- [Секция INTERSECT](intersect.md) +- [Секция EXCEPT](except.md) - [Секция INTO OUTFILE](into-outfile.md) - [Секция FORMAT](format.md) diff --git a/docs/ru/sql-reference/statements/select/intersect.md b/docs/ru/sql-reference/statements/select/intersect.md new file mode 100644 index 00000000000..def2ff8b9f9 --- /dev/null +++ b/docs/ru/sql-reference/statements/select/intersect.md @@ -0,0 +1,73 @@ +--- +toc_title: INTERSECT +--- + +# INTERSECT Clause {#intersect-clause} + +Оператор `INTERSECT` возвращает только те строки, которые есть только в результатах первого и второго запросов. В запросах должны совпадать количество столбцов, их порядок и тип. Результат `INTERSECT` может содержать повторяющиеся строки. + +Если используется несколько операторов `INTERSECT` и скобки не указаны, пересечение выполняется слева направо. У оператора `INTERSECT` более высокий приоритет выполнения, чем у `UNION` и `EXCEPT`. + + +``` sql +SELECT column1 [, column2 ] +FROM table1 +[WHERE condition] + +INTERSECT + +SELECT column1 [, column2 ] +FROM table2 +[WHERE condition] + +``` +Условие может быть любым в зависимости от ваших требований. + +**Примеры** + +Запрос: + +``` sql +SELECT number FROM numbers(1,10) INTERSECT SELECT number FROM numbers(3,6); +``` + +Результат: + +``` text +┌─number─┐ +│ 3 │ +│ 4 │ +│ 5 │ +│ 6 │ +│ 7 │ +│ 8 │ +└────────┘ +``` + +Запрос: + +``` sql +CREATE TABLE t1(one String, two String, three String) ENGINE=Memory(); +CREATE TABLE t2(four String, five String, six String) ENGINE=Memory(); + +INSERT INTO t1 VALUES ('q', 'm', 'b'), ('s', 'd', 'f'), ('l', 'p', 'o'), ('s', 'd', 'f'), ('s', 'd', 'f'), ('k', 't', 'd'), ('l', 'p', 'o'); +INSERT INTO t2 VALUES ('q', 'm', 'b'), ('b', 'd', 'k'), ('s', 'y', 't'), ('s', 'd', 'f'), ('m', 'f', 'o'), ('k', 'k', 'd'); + +SELECT * FROM t1 INTERSECT SELECT * FROM t2; +``` + +Результат: + +``` text +┌─one─┬─two─┬─three─┐ +│ q │ m │ b │ +│ s │ d │ f │ +│ s │ d │ f │ +│ s │ d │ f │ +└─────┴─────┴───────┘ +``` + +**См. также** + +- [UNION](union.md#union-clause) +- [EXCEPT](except.md#except-clause) From 82fc693d55871467439b888db0d19463e87025be Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 22 Oct 2021 22:38:34 +0300 Subject: [PATCH 054/173] russian fixes --- docs/en/sql-reference/statements/select/except.md | 2 +- docs/ru/sql-reference/statements/select/except.md | 6 +++--- docs/ru/sql-reference/statements/select/intersect.md | 6 +++--- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/statements/select/except.md b/docs/en/sql-reference/statements/select/except.md index d45c0556a1a..e6d9b365a91 100644 --- a/docs/en/sql-reference/statements/select/except.md +++ b/docs/en/sql-reference/statements/select/except.md @@ -66,4 +66,4 @@ Result: **See Also** - [UNION](union.md#union-clause) -- [INTERSECT](except.md#except-clause) +- [INTERSECT](intersect.md#intersect-clause) diff --git a/docs/ru/sql-reference/statements/select/except.md b/docs/ru/sql-reference/statements/select/except.md index 6929eff1c81..eb4986aceea 100644 --- a/docs/ru/sql-reference/statements/select/except.md +++ b/docs/ru/sql-reference/statements/select/except.md @@ -4,9 +4,9 @@ toc_title: EXCEPT # Секция EXCEPT {#except-clause} -Оператор `EXCEPT` возвращает только те строки, которые являются результатом первого запроса без результатов второго. В запросах количество, порядок следования и типы столбцов должны совпадать. Результат `EXCEPT` может содержать повторяющиеся строки. +`EXCEPT` возвращает только те строки, которые являются результатом первого запроса без результатов второго. В запросах количество, порядок следования и типы столбцов должны совпадать. Результат `EXCEPT` может содержать повторяющиеся строки. -Если используется несколько `EXCEPT`, и в выражении не указаны скобки, `EXCEPT` выполняется по порядку слева направо. Оператор `EXCEPT` имеет такой же приоритет выполнения, как `UNION`, и ниже, чем `INTERSECT`. +Если используется несколько `EXCEPT`, и в выражении не указаны скобки, `EXCEPT` выполняется по порядку слева направо. `EXCEPT` имеет такой же приоритет выполнения, как `UNION`, и приоритет ниже, чем у `INTERSECT`. ``` sql SELECT column1 [, column2 ] @@ -66,4 +66,4 @@ SELECT * FROM t1 EXCEPT SELECT * FROM t2; **См. также** - [UNION](union.md#union-clause) -- [INTERSECT](except.md#except-clause) \ No newline at end of file +- [INTERSECT](intersect.md#intersect-clause) \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/select/intersect.md b/docs/ru/sql-reference/statements/select/intersect.md index def2ff8b9f9..cf0c5aee55a 100644 --- a/docs/ru/sql-reference/statements/select/intersect.md +++ b/docs/ru/sql-reference/statements/select/intersect.md @@ -2,11 +2,11 @@ toc_title: INTERSECT --- -# INTERSECT Clause {#intersect-clause} +# Секция INTERSECT {#intersect-clause} -Оператор `INTERSECT` возвращает только те строки, которые есть только в результатах первого и второго запросов. В запросах должны совпадать количество столбцов, их порядок и тип. Результат `INTERSECT` может содержать повторяющиеся строки. +`INTERSECT` возвращает только те строки, которые есть только в результатах первого и второго запросов. В запросах должны совпадать количество столбцов, их порядок и тип. Результат `INTERSECT` может содержать повторяющиеся строки. -Если используется несколько операторов `INTERSECT` и скобки не указаны, пересечение выполняется слева направо. У оператора `INTERSECT` более высокий приоритет выполнения, чем у `UNION` и `EXCEPT`. +Если используется несколько `INTERSECT` и скобки не указаны, пересечение выполняется слева направо. У `INTERSECT` более высокий приоритет выполнения, чем у `UNION` и `EXCEPT`. ``` sql From 258f8cb2f7ccd22554c8c050fe265120284eeac4 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 22 Oct 2021 23:10:02 +0300 Subject: [PATCH 055/173] Add normalizeUTF8 functions. --- .../functions/string-functions.md | 126 +++++++++++++++++- 1 file changed, 125 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 4ac3a175aba..f6dbadb875a 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -826,11 +826,135 @@ normalizeUTF8NFKD(words) **Returned value** -- String transformed to normalization form C (NFC). +- String transformed to NFC normalization form. Type: [String](../../sql-reference/data-types/string.md). +**Example** +Query: + +``` sql +SELECT length('ё'), normalizeUTF8NFC('ё') AS nfc, length(nfc) AS nfc_len; +``` + +Result: + +``` text +┌─length('ё')─┬─nfc─┬─nfc_len─┐ +│ 2 │ ё │ 2 │ +└─────────────┴─────┴─────────┘ +``` + +## normalizeUTF8NFD {#normalizeutf8nfd} + +Converts a string to [NFD normalized form](https://en.wikipedia.org/wiki/Unicode_equivalence#Normal_forms), assuming the string contains a set of bytes that make up a UTF-8 encoded text. + +**Syntax** + +``` sql +normalizeUTF8NFD(words) +``` + +**Arguments** + +- `words` — Input string that contains UTF-8 encoded text. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- String transformed to NFD normalization form. + +Type: [String](../../sql-reference/data-types/string.md). + +**Example** + +Query: + +``` sql +SELECT length('ё'), normalizeUTF8NFD('ё') AS nfd, length(nfd) AS nfd_len; +``` + +Result: + +``` text +┌─length('ё')─┬─nfd─┬─nfd_len─┐ +│ 2 │ ё │ 4 │ +└─────────────┴─────┴─────────┘ +``` + +## normalizeUTF8NFKC {#normalizeutf8nfkc} + +Converts a string to [NFKC normalized form](https://en.wikipedia.org/wiki/Unicode_equivalence#Normal_forms), assuming the string contains a set of bytes that make up a UTF-8 encoded text. + +**Syntax** + +``` sql +normalizeUTF8NFKC(words) +``` + +**Arguments** + +- `words` — Input string that contains UTF-8 encoded text. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- String transformed to NFKC normalization form. + +Type: [String](../../sql-reference/data-types/string.md). + +**Example** + +Query: + +``` sql +SELECT length('ё'), normalizeUTF8NFKC('ё') AS nfkc, length(nfkc) AS nfkc_len; +``` + +Result: + +``` text +┌─length('ё')─┬─nfkc─┬─nfkc_len─┐ +│ 2 │ ё │ 2 │ +└─────────────┴──────┴──────────┘ + +``` + +## normalizeUTF8NFKD {#normalizeutf8nfkd} + + +Converts a string to [NFKD normalized form](https://en.wikipedia.org/wiki/Unicode_equivalence#Normal_forms), assuming the string contains a set of bytes that make up a UTF-8 encoded text. + +**Syntax** + +``` sql +normalizeUTF8NFKD(words) +``` + +**Arguments** + +- `words` — Input string that contains UTF-8 encoded text. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- String transformed to NFKD normalization form. + +Type: [String](../../sql-reference/data-types/string.md). + +**Example** + +Query: + +``` sql +SELECT length('ё'), normalizeUTF8NFKD('ё') AS nfkd, length(nfkd) AS nfkd_len; +``` + +Result: + +``` text +┌─length('ё')─┬─nfkd─┬─nfkd_len─┐ +│ 2 │ ё │ 4 │ +└─────────────┴──────┴──────────┘ +``` ## encodeXMLComponent {#encode-xml-component} From 99800cc8d4f529c09ba25d3708645b08613e4990 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Sat, 23 Oct 2021 11:28:22 +0800 Subject: [PATCH 056/173] Translation for Continuous Integration Checks --- docs/zh/development/continuous-integration.md | 113 ++++++++++++------ 1 file changed, 76 insertions(+), 37 deletions(-) diff --git a/docs/zh/development/continuous-integration.md b/docs/zh/development/continuous-integration.md index 205a7edf15e..7e4c1bf36a9 120000 --- a/docs/zh/development/continuous-integration.md +++ b/docs/zh/development/continuous-integration.md @@ -1,9 +1,9 @@ # 持续集成检查 {#continuous-integration-checks} -当你提交一个pull请求时, ClickHouse的[持续集成(CI)系统](../tests.md#test-automation)会对您的代码运行一些自动检查. +当你提交一个pull请求时, ClickHouse[持续集成(CI)系统](../tests.md#test-automation)会对您的代码运行一些自动检查. 这在存储库维护者(来自ClickHouse团队的人)筛选了您的代码并将可测试标签添加到您的pull请求之后发生. -检查的结果被列在[GitHub检查文档](https://docs.github.com/en/github/collaborating-with-pull-requests/collaborating-on-repositories-with-code-quality-features/about-status-checks)中描述的GitHub pull请求页面. +检查的结果被列在[GitHub检查文档](https://docs.github.com/en/github/collaborating-with-pull-requests/collaborating-on-repositories-with-code-quality-features/about-status-checks)中所述的GitHub pull请求页面. 如果检查失败,您可能被要求去修复它. 该界面介绍了您可能遇到的检查,以及如何修复它们. @@ -17,49 +17,48 @@ git push 如果您不确定要做什么,可以向维护人员寻求帮助. - # 与Master合并 {#merge-with-master} -验证PR是否可以合并到master. 如果没有, 它将返回消息为'Cannot fetch mergecommit'的失败.要修复这个检查, 解决[GitHub文档](https://docs.github.com/en/github/collaborating-with-pull-requests/addressing-merge-conflicts/resolving-a-merge-conflict-on-github)中描述的冲突, 或者使用git合并主分支到你的pull request分支. +验证PR是否可以合并到master. 如果没有, 它将失败并显示消息'Cannot fetch mergecommit'的.请按[GitHub文档](https://docs.github.com/en/github/collaborating-with-pull-requests/addressing-merge-conflicts/resolving-a-merge-conflict-on-github)中描述的冲突解决, 或使用git将主分支合并到您的pull请求分支来修复这个检查. # 文档检查 {#docs-check} 尝试构建ClickHouse文档网站. 如果您更改了文档中的某些内容, 它可能会失败. 最可能的原因是文档中的某些交叉链接是错误的. 转到检查报告并查找`ERROR`和`WARNING`消息. -# 查看报告详情 {#report-details} -- [状态页案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) -- `docs_output.txt`记录了build日志信息. [成功结果案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check/docs_output.txt) +# 报告详情 {#report-details} +- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) +- `docs_output.txt`包含构建日志信息. [成功结果案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check/docs_output.txt) # 描述信息检查 {#description-check} -检查pull请求的描述是否符合[PULL_REQUEST_TEMPLATE.md](https://github.com/ClickHouse/ClickHouse/blob/master/.github/PULL_REQUEST_TEMPLATE.md)模板. +检查pull请求的描述是否符合[PULL_REQUEST_TEMPLATE.md](../../../.github/PULL_REQUEST_TEMPLATE.md)模板. -您必须为您的更改指定一个更改日志类别(例如,Bug修复), 并且为[CHANGELOG.md](https://clickhouse.com/docs/en/whats-new/changelog/)编写一条用户可读的消息用来描述更改. +您必须为您的更改指定一个更改日志类别(例如,Bug修复), 并且为[CHANGELOG.md](../whats-new/changelog/)编写一条用户可读的消息用来描述更改. # 推送到DockerHub {#push-to-dockerhub} 生成用于构建和测试的docker映像, 然后将它们推送到DockerHub. # 标记检查 {#marker-check} -该检查意味着集成系统已经开始处理PR.当它处于待定状态时, 意味着还没有开始所有的检查. 在所有检查启动后,它将状态更改为'成功'. +该检查意味着CI系统已经开始处理PR.当它处于'待处理'状态时,意味着尚未开始所有检查. 启动所有检查后,状态更改为'成功'. # 格式检查 {#style-check} 使用`utils/check-style/check-style`二进制文件执行一些简单的基于正则表达式的代码样式检查(注意, 它可以在本地运行). -如果失败, 按照[代码样式指南](https://clickhouse.com/docs/en/development/style/)修复样式错误. +如果失败, 按照[代码样式指南](./style.md)修复样式错误. -# 查看报告详情 {#report-details} -- [状态页案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) -- `docs_output.txt`记录了查结果错误(无效表格等), 空白页表示没有错误. [成功结果案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check/docs_output.txt) +# 报告详情 {#report-details} +- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check.html) +- `docs_output.txt`记录了查结果错误(无效表格等), 空白页表示没有错误. [成功结果案例](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check/output.txt) -# PVS Check {#pvs-check} -使用静态分析工具[PVS-studio](https://www.viva64.com/en/pvs-studio/)检查代码. 查看报告找出确切的错误.如果可以则修复它们, 如果不行, 可以请ClickHouse的维护人员帮忙. +# PVS 检查 {#pvs-check} +使用静态分析工具[PVS-studio](https://www.viva64.com/en/pvs-studio/)检查代码. 查看报告以查看确切的错误.如果可以则修复它们, 如果不行, 可以向ClickHouse的维护人员寻求帮忙. -# 查看报告详情 {#report-details} -- [状态页案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) -- `test_run.txt.out.log`记录了构建和分析日志文件.它只包含解析或未找到的错误. -- `HTML report`记录了分析结果.关于它的描述, 请访问PVS的[官方网站](https://www.viva64.com/en/m/0036/#ID14E9A2B2CD) +# 报告详情 {#report-details} +- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/pvs_check.html) +- `test_run.txt.out.log`包含构建和分析日志文件.它只包含解析或未找到的错误. +- `HTML report`包含分析结果.有关说明请访问PVS的[官方网站](https://www.viva64.com/en/m/0036/#ID14E9A2B2CD) -# {#fast-test} -通常情况下这是PR运行的第一个检查.它构建ClickHouse以及大多数无状态运行测试, 其中省略了一些.如果失败,在修复之前不会开始进一步的检查. 通过报告查看哪些测试失败, 然后按照[这里](https://clickhouse.com/docs/en/development/tests/#functional-test-locally)描述的在本地重现失败. +# 快速测试 {#fast-test} +通常情况下这是PR运行的第一个检查.它构建ClickHouse以及大多数无状态运行测试, 其中省略了一些.如果失败,在修复之前不会开始进一步的检查. 查看报告以了解哪些测试失败, 然后按照[此处](./tests.md#functional-test-locally)描述的在本地重现失败. -# 查看报告详情 {#report-details} -[状态页案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) +# 报告详情 {#report-details} +[状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/fast_test.html) # 状态页文件 {#status-page-files} - `runlog.out.log` 是包含所有其他日志的通用日志. @@ -72,27 +71,27 @@ git push - `install_log.txt` - `clickhouse-server.err.log` - `build_log.txt` -- `cmake_log.txt` 包含关于C/ c++和Linux标志检查的消息. +- `cmake_log.txt` 包含关于C/C++和Linux标志检查的消息. # 状态页列信息 {#status-page-columns} -- 测试名称 -- 包含测试的名称(没有路径, 例如, 所有类型的测试将被剥离到该名称). -- 测试状态 -- 跳过、成功或失败. -- 测试时间, 等等. -- 这个测试是空的. +- 测试名称 -- 包含测试的名称(不带路径, 例如, 所有类型的测试将被剥离到该名称). +- 测试状态 -- 跳过、成功或失败之一. +- 测试时间, 秒. -- 这个测试是空的. # 建构检查 {#build-check} -在各种配置中构建ClickHouse, 以便在后续步骤中使用. 您必须修复失败的构建.构建日志通常有足够的信息来修复错误, 但是您可能必须在本地重新生成错误. cmake选项可以在构建日志中找到, 为cmake做准备.使用这些选项并遵循一般的构建过程. +在各种配置中构建ClickHouse, 以便在后续步骤中使用. 您必须修复失败的构建.构建日志通常有足够的信息来修复错误, 但是您可能必须在本地重现故障. `cmake`选项可以在构建日志中通过grep `cmake`操作找到.使用这些选项并遵循[一般的构建过程](./build.md). -# 查看报告详情 {#report-details} -[状态页案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) -- **Compiler**: `gcc-9` or `clang-10` (or `clang-10-xx` for other architectures e.g. `clang-10-freebsd`). +# 报告详情 {#report-details} +[状态页示例](https://clickhouse-builds.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/clickhouse_build_check/report.html) +- **Compiler**: `gcc-9` 或 `clang-10` (或其他架构的`clang-10-xx`, 比如`clang-10-freebsd`). - **Build type**: `Debug` or `RelWithDebInfo` (cmake). - **Sanitizer**: `none` (without sanitizers), `address` (ASan), `memory` (MSan), `undefined` (UBSan), or `thread` (TSan). - **Bundled**: `bundled` 构建使用来自 `contrib` 库, 而 `unbundled` 构建使用系统库. - **Splitted**: `splitted` is a [split build](https://clickhouse.com/docs/en/development/build/#split-build) - **Status**: `成功` 或 `失败` -- **Build log**: 链接到生成和文件复制日志,这在生成失败时很有用. +- **Build log**: 链接到构建和文件复制日志, 当构建失败时很有用. - **Build time**. -- **Artifacts**: 构建结果文件 (with `XXX` being the server version e.g. `20.8.1.4344`). +- **Artifacts**: 构建结果文件 (`XXX`是服务器版本, 比如`20.8.1.4344`). - `clickhouse-client_XXX_all.deb` -` clickhouse-common-static-dbg_XXX[+asan, +msan, +ubsan, +tsan]_amd64.deb` - `clickhouse-common-staticXXX_amd64.deb` @@ -102,15 +101,55 @@ git push - `clickhouse_XXX_amd64.changes` - `clickhouse`: Main built binary. - `clickhouse-odbc-bridge` - - `unit_tests_dbms`: GoogleTest二进制和ClickHouse单元测试. + - `unit_tests_dbms`: 带有 ClickHouse 单元测试的 GoogleTest 二进制文件. - `shared_build.tgz`: 使用共享库构建. - `performance.tgz`: 用于性能测试的特殊包. -# 特殊的构建检查 {#special-buildcheck} -使用clang-tidy执行静态分析和代码风格检查. 报告类似于构建检查. 修复在构建日志中发现的错误. +# 特殊构建检查 {#special-buildcheck} +使用clang-tidy执行静态分析和代码样式检查. 该报告类似于构建检查. 修复在构建日志中发现的错误. +# 功能无状态测试 {#functional-stateless-tests} +为构建在不同配置中的ClickHouse二进制文件运行[无状态功能测试](./tests.md#functional-tests)——发布、调试、使用杀毒软件等.通过报告查看哪些测试失败,然后按照[此处](./tests.md#functional-test-locally)描述的在本地重现失败.注意, 您必须使用正确的构建配置来重现——在AddressSanitizer下测试可能失败,但在Debug中可以通过.从[CI构建检查页面](./build.md#you-dont-have-to-build-clickhouse)下载二进制文件, 或者在本地构建它. +# 功能有状态测试 {#functional-stateful-tests} +运行[有状态功能测试](./tests.md#functional-tests).以无状态功能测试相同的方式对待它们.不同之处在于它们需要从[Yandex.Metrica数据集](https://clickhouse.com/docs/en/getting-started/example-datasets/metrica/)的`hits`和`visits`表来运行. +# 集成测试 {#integration-tests} +运行[集成测试](./tests.md#integration-tests). +# Testflows 检查{#testflows-check} +使用Testflows测试系统去运行一些测试, 在[此处](../../../tests/testflows/README.md#running-tests-locally)查看如何在本地运行它们. +# 压力测试 {#stress-test} +从多个客户端并发运行无状态功能测试, 用以检测与并发相关的错误.如果失败: +``` +* Fix all other test failures first; +* Look at the report to find the server logs and check them for possible causes + of error. +``` +# 冒烟测试 {#split-build-smoke-test} +检查[拆分构建](./build.md#split-build)配置中的服务器构建是否可以启动并运行简单查询.如果失败: +``` +* Fix other test errors first; +* Build the server in [split build](./build.md#split-build) configuration + locally and check whether it can start and run `select 1`. +``` + +# 兼容性检查 {#compatibility-check} +检查`clickhouse`二进制文件是否可以在带有旧libc版本的发行版上运行.如果失败, 请向维护人员寻求帮助. + +# AST模糊器 {#ast-fuzzer} +运行随机生成的查询来捕获程序错误.如果失败, 请向维护人员寻求帮助. + +# 性能测试 {#performance-tests} +测量查询性能的变化. 这是最长的检查, 只需不到 6 小时即可运行.性能测试报告在[此处](../../../docker/test/performance-comparison/README.md#how-to-read-the-report)有详细描述. + +# 质量保证 {#qa} +什么是状态页面上的任务(专用网络)项目? + +它是 Yandex 内部工作系统的链接. Yandex 员工可以看到检查的开始时间及其更详细的状态. + +运行测试的地方 + +Yandex 内部基础设施的某个地方. From 12b378f27f1da3e401bb6deab183d06a35bb446f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Sat, 23 Oct 2021 15:56:25 +0800 Subject: [PATCH 057/173] Translation for Kerberos Document: Chinese translation for Operations - External User Authenticators and Directories - Kerberos --- .../external-authenticators/kerberos.md | 106 +++++++++++++++++- 1 file changed, 105 insertions(+), 1 deletion(-) diff --git a/docs/zh/operations/external-authenticators/kerberos.md b/docs/zh/operations/external-authenticators/kerberos.md index b5a4d557de4..fb7a6c0d67a 120000 --- a/docs/zh/operations/external-authenticators/kerberos.md +++ b/docs/zh/operations/external-authenticators/kerberos.md @@ -1 +1,105 @@ -../../../en/operations/external-authenticators/kerberos.md \ No newline at end of file +# Kerberos认证 {#kerberos} +现有正确配置的 ClickHouse 用户可以通过 Kerberos 身份验证协议进行身份验证. + +目前, Kerberos 只能用作现有用户的外部身份验证器,这些用户在 `users.xml` 或本地访问控制路径中定义. +这些用户只能使用 HTTP 请求, 并且必须能够使用 GSS-SPNEGO 机制进行身份验证. + +对于这种方法, 必须在系统中配置 Kerberos, 且必须在 ClickHouse 配置中启用. + +# 开启Kerberos {#enabling-kerberos-in-clickHouse} +要启用 Kerberos, 应该在 `config.xml` 中包含 `kerberos` 部分. 此部分可能包含其他参数. + +## 参数: {#parameters} +- `principal` - 将在接受安全上下文时获取和使用的规范服务主体名称. +- 此参数是可选的, 如果省略, 将使用默认主体. + +- `realm` - 一个领域, 用于将身份验证限制为仅那些发起者领域与其匹配的请求. + +- 此参数是可选的,如果省略,则不会应用其他领域的过滤. + +示例 (进入 `config.xml`): +``` + + + + +``` + +主体规范: +``` + + + + HTTP/clickhouse.example.com@EXAMPLE.COM + + +``` + +按领域过滤: +``` + + + + EXAMPLE.COM + + +``` + +**注意** + +您只能定义一个 `kerberos` 部分. 多个 `kerberos` 部分的存在将强制 ClickHouse 禁用 Kerberos 身份验证. + +**注意** + +`主体`和`领域`部分不能同时指定. `主体`和`领域`的出现将迫使ClickHouse禁用Kerberos身份验证. + +# Kerberos作为现有用户的外部身份验证器 {#kerberos-as-an-external-authenticator-for-existing-users} +Kerberos可以用作验证本地定义用户(在`users.xml`或本地访问控制路径中定义的用户)身份的方法。目前,**只有**通过HTTP接口的请求才能被认证(通过GSS-SPNEGO机制). + +Kerberos主体名称格式通常遵循以下模式: +- *primary/instance@REALM* + +*/instance* 部分可能出现零次或多次. **发起者的规范主体名称的主要部分应与被认证用户名匹配, 以便身份验证成功**. + +# `users.xml`中启用Kerberos {#enabling-kerberos-in-users.xml} +为了启用用户的 Kerberos 身份验证, 请在用户定义中指定 `kerberos` 部分而不是`密码`或类似部分. + +参数: +- `realm` - 用于将身份验证限制为仅那些发起者的领域与其匹配的请求的领域. +- 此参数是可选的, 如果省略, 则不会应用其他按领域的过滤. + +示例 (进入 `users.xml`): +``` + + + + + + + + EXAMPLE.COM + + + + +``` + +**警告** + +注意, Kerberos身份验证不能与任何其他身份验证机制一起使用. 任何其他部分(如`密码`和`kerberos`)的出现都会迫使ClickHouse关闭. + +**提醒** + +请注意, 现在, 一旦用户 `my_user` 使用 `kerberos`, 必须在主 `config.xml` 文件中启用 Kerberos,如前所述. + +# 使用 SQL 启用 Kerberos {#enabling-kerberos-using-sql} +在 ClickHouse 中启用 [SQL 驱动的访问控制和帐户管理](./access-rights.md#access-control)后, 也可以使用 SQL 语句创建由 Kerberos 识别的用户. + +``` +CREATE USER my_user IDENTIFIED WITH kerberos REALM 'EXAMPLE.COM' +``` + +...或者, 不按领域过滤: +``` +CREATE USER my_user IDENTIFIED WITH kerberos +``` From 3e6870c54004b8d1a76263078a19b8700a7bf806 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Sat, 23 Oct 2021 16:24:43 +0800 Subject: [PATCH 058/173] chinese translation MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Adjust 事件时间 for DateTime64 to 事件时间(微秒) --- docs/zh/operations/system-tables/asynchronous_metric_log.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/zh/operations/system-tables/asynchronous_metric_log.md b/docs/zh/operations/system-tables/asynchronous_metric_log.md index d49676794fc..592fb99c5ef 100644 --- a/docs/zh/operations/system-tables/asynchronous_metric_log.md +++ b/docs/zh/operations/system-tables/asynchronous_metric_log.md @@ -10,7 +10,7 @@ machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 列: - `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)) — 指标值. @@ -35,4 +35,4 @@ SELECT * FROM system.asynchronous_metric_log LIMIT 10 **另请参阅** - [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — 包含在后台定期计算的指标. -- [system.metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — 包含定期刷新到磁盘表 `system.metrics` 以及 `system.events` 中的指标值历史记录. \ No newline at end of file +- [system.metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — 包含定期刷新到磁盘表 `system.metrics` 以及 `system.events` 中的指标值历史记录. From 4fa3b94eada854dbd8818d17bc45cd99e1c2a942 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Sun, 24 Oct 2021 20:29:41 +0300 Subject: [PATCH 059/173] Update intersect.md --- docs/ru/sql-reference/statements/select/intersect.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/select/intersect.md b/docs/ru/sql-reference/statements/select/intersect.md index cf0c5aee55a..2206b141472 100644 --- a/docs/ru/sql-reference/statements/select/intersect.md +++ b/docs/ru/sql-reference/statements/select/intersect.md @@ -4,7 +4,7 @@ toc_title: INTERSECT # Секция INTERSECT {#intersect-clause} -`INTERSECT` возвращает только те строки, которые есть только в результатах первого и второго запросов. В запросах должны совпадать количество столбцов, их порядок и тип. Результат `INTERSECT` может содержать повторяющиеся строки. +`INTERSECT` возвращает строки, которые есть только в результатах первого и второго запросов. В запросах должны совпадать количество столбцов, их порядок и тип. Результат `INTERSECT` может содержать повторяющиеся строки. Если используется несколько `INTERSECT` и скобки не указаны, пересечение выполняется слева направо. У `INTERSECT` более высокий приоритет выполнения, чем у `UNION` и `EXCEPT`. From 37de54937cede70f5b7ce4fd6eb8a65c6c6691af Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sun, 24 Oct 2021 22:32:28 +0300 Subject: [PATCH 060/173] fix use-after-free --- src/Interpreters/AsynchronousInsertQueue.cpp | 10 ++++++++-- ...ert_7.reference => 02015_async_inserts_7.reference} | 0 ...2015_async_insert_7.sh => 02015_async_inserts_7.sh} | 0 3 files changed, 8 insertions(+), 2 deletions(-) rename tests/queries/0_stateless/{02015_async_insert_7.reference => 02015_async_inserts_7.reference} (100%) rename tests/queries/0_stateless/{02015_async_insert_7.sh => 02015_async_inserts_7.sh} (100%) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index cc04c5f013f..2f86c8bef28 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -407,14 +407,20 @@ try } StreamingFormatExecutor executor(header, format, std::move(on_error), std::move(adding_defaults_transform)); - std::unique_ptr buffer; + std::unique_ptr last_buffer; for (const auto & entry : data->entries) { - buffer = std::make_unique(entry->bytes); + auto buffer = std::make_unique(entry->bytes); current_entry = entry; total_rows += executor.execute(*buffer); + + /// Keep buffer, because it still can be used + /// in destructor, while resetting buffer at next iteration. + last_buffer = std::move(buffer); } + format->addBuffer(std::move(last_buffer)); + auto chunk = Chunk(executor.getResultColumns(), total_rows); size_t total_bytes = chunk.bytes(); diff --git a/tests/queries/0_stateless/02015_async_insert_7.reference b/tests/queries/0_stateless/02015_async_inserts_7.reference similarity index 100% rename from tests/queries/0_stateless/02015_async_insert_7.reference rename to tests/queries/0_stateless/02015_async_inserts_7.reference diff --git a/tests/queries/0_stateless/02015_async_insert_7.sh b/tests/queries/0_stateless/02015_async_inserts_7.sh similarity index 100% rename from tests/queries/0_stateless/02015_async_insert_7.sh rename to tests/queries/0_stateless/02015_async_inserts_7.sh From fb4eb4cf2362163e12e61c05083a289b63d39491 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Mon, 25 Oct 2021 11:45:24 +0800 Subject: [PATCH 061/173] chinese translation adjust title level --- .../external-authenticators/kerberos.md | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/docs/zh/operations/external-authenticators/kerberos.md b/docs/zh/operations/external-authenticators/kerberos.md index fb7a6c0d67a..986795aea8b 120000 --- a/docs/zh/operations/external-authenticators/kerberos.md +++ b/docs/zh/operations/external-authenticators/kerberos.md @@ -1,4 +1,4 @@ -# Kerberos认证 {#kerberos} +# Kerberos认证 {#external-authenticators-kerberos} 现有正确配置的 ClickHouse 用户可以通过 Kerberos 身份验证协议进行身份验证. 目前, Kerberos 只能用作现有用户的外部身份验证器,这些用户在 `users.xml` 或本地访问控制路径中定义. @@ -6,10 +6,10 @@ 对于这种方法, 必须在系统中配置 Kerberos, 且必须在 ClickHouse 配置中启用. -# 开启Kerberos {#enabling-kerberos-in-clickHouse} +## 开启Kerberos {#enabling-kerberos-in-clickHouse} 要启用 Kerberos, 应该在 `config.xml` 中包含 `kerberos` 部分. 此部分可能包含其他参数. -## 参数: {#parameters} +#### 参数: {#parameters} - `principal` - 将在接受安全上下文时获取和使用的规范服务主体名称. - 此参数是可选的, 如果省略, 将使用默认主体. @@ -18,7 +18,7 @@ - 此参数是可选的,如果省略,则不会应用其他领域的过滤. 示例 (进入 `config.xml`): -``` +```xml @@ -26,7 +26,7 @@ ``` 主体规范: -``` +```xml @@ -36,7 +36,7 @@ ``` 按领域过滤: -``` +```xml @@ -53,7 +53,7 @@ `主体`和`领域`部分不能同时指定. `主体`和`领域`的出现将迫使ClickHouse禁用Kerberos身份验证. -# Kerberos作为现有用户的外部身份验证器 {#kerberos-as-an-external-authenticator-for-existing-users} +## Kerberos作为现有用户的外部身份验证器 {#kerberos-as-an-external-authenticator-for-existing-users} Kerberos可以用作验证本地定义用户(在`users.xml`或本地访问控制路径中定义的用户)身份的方法。目前,**只有**通过HTTP接口的请求才能被认证(通过GSS-SPNEGO机制). Kerberos主体名称格式通常遵循以下模式: @@ -61,7 +61,7 @@ Kerberos主体名称格式通常遵循以下模式: */instance* 部分可能出现零次或多次. **发起者的规范主体名称的主要部分应与被认证用户名匹配, 以便身份验证成功**. -# `users.xml`中启用Kerberos {#enabling-kerberos-in-users.xml} +### `users.xml`中启用Kerberos {#enabling-kerberos-in-users-xml} 为了启用用户的 Kerberos 身份验证, 请在用户定义中指定 `kerberos` 部分而不是`密码`或类似部分. 参数: @@ -92,14 +92,14 @@ Kerberos主体名称格式通常遵循以下模式: 请注意, 现在, 一旦用户 `my_user` 使用 `kerberos`, 必须在主 `config.xml` 文件中启用 Kerberos,如前所述. -# 使用 SQL 启用 Kerberos {#enabling-kerberos-using-sql} +### 使用 SQL 启用 Kerberos {#enabling-kerberos-using-sql} 在 ClickHouse 中启用 [SQL 驱动的访问控制和帐户管理](./access-rights.md#access-control)后, 也可以使用 SQL 语句创建由 Kerberos 识别的用户. -``` +```sql CREATE USER my_user IDENTIFIED WITH kerberos REALM 'EXAMPLE.COM' ``` ...或者, 不按领域过滤: -``` +```sql CREATE USER my_user IDENTIFIED WITH kerberos ``` From 171c5845ed7c84dbacdb242eab4c38fbfddc156f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Mon, 25 Oct 2021 14:13:03 +0800 Subject: [PATCH 062/173] chinese translation adjust waning module --- docs/zh/operations/external-authenticators/kerberos.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/zh/operations/external-authenticators/kerberos.md b/docs/zh/operations/external-authenticators/kerberos.md index 986795aea8b..1121a86a793 120000 --- a/docs/zh/operations/external-authenticators/kerberos.md +++ b/docs/zh/operations/external-authenticators/kerberos.md @@ -45,11 +45,11 @@ ``` -**注意** +!!! warning "注意" 您只能定义一个 `kerberos` 部分. 多个 `kerberos` 部分的存在将强制 ClickHouse 禁用 Kerberos 身份验证. -**注意** +!!! warning "注意" `主体`和`领域`部分不能同时指定. `主体`和`领域`的出现将迫使ClickHouse禁用Kerberos身份验证. @@ -84,11 +84,11 @@ Kerberos主体名称格式通常遵循以下模式: ``` -**警告** +!!! warning "警告" 注意, Kerberos身份验证不能与任何其他身份验证机制一起使用. 任何其他部分(如`密码`和`kerberos`)的出现都会迫使ClickHouse关闭. -**提醒** +!!! info "提醒" 请注意, 现在, 一旦用户 `my_user` 使用 `kerberos`, 必须在主 `config.xml` 文件中启用 Kerberos,如前所述. From db47dcf14fdd4122ba06b3ab6479a13ebe7b8930 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 25 Oct 2021 11:18:00 +0300 Subject: [PATCH 063/173] Added test. --- ...er_table_alter_block_structure_2.reference | 3 +++ ...6_buffer_table_alter_block_structure_2.sql | 25 +++++++++++++++++++ 2 files changed, 28 insertions(+) create mode 100644 tests/queries/0_stateless/01506_buffer_table_alter_block_structure_2.reference create mode 100644 tests/queries/0_stateless/01506_buffer_table_alter_block_structure_2.sql diff --git a/tests/queries/0_stateless/01506_buffer_table_alter_block_structure_2.reference b/tests/queries/0_stateless/01506_buffer_table_alter_block_structure_2.reference new file mode 100644 index 00000000000..1f90610041b --- /dev/null +++ b/tests/queries/0_stateless/01506_buffer_table_alter_block_structure_2.reference @@ -0,0 +1,3 @@ +2020-01-01 00:05:00 +2020-01-01 00:05:00 +2020-01-01 00:06:00 hello diff --git a/tests/queries/0_stateless/01506_buffer_table_alter_block_structure_2.sql b/tests/queries/0_stateless/01506_buffer_table_alter_block_structure_2.sql new file mode 100644 index 00000000000..8862037c82b --- /dev/null +++ b/tests/queries/0_stateless/01506_buffer_table_alter_block_structure_2.sql @@ -0,0 +1,25 @@ +DROP TABLE IF EXISTS buf_dest; +DROP TABLE IF EXISTS buf; + +CREATE TABLE buf_dest (timestamp DateTime) +ENGINE = MergeTree PARTITION BY toYYYYMMDD(timestamp) +ORDER BY (timestamp); + +CREATE TABLE buf (timestamp DateTime) Engine = Buffer(currentDatabase(), buf_dest, 16, 0.1, 0.1, 2000000, 20000000, 100000000, 300000000);; + +INSERT INTO buf (timestamp) VALUES (toDateTime('2020-01-01 00:05:00')); + +--- wait for buffer to flush +SELECT sleep(1) from numbers(1) settings max_block_size=1 format Null; + +ALTER TABLE buf_dest ADD COLUMN s String; +ALTER TABLE buf ADD COLUMN s String; + +SELECT * FROM buf; + +INSERT INTO buf (timestamp, s) VALUES (toDateTime('2020-01-01 00:06:00'), 'hello'); + +SELECT * FROM buf ORDER BY timestamp; + +DROP TABLE IF EXISTS buf; +DROP TABLE IF EXISTS buf_dest; From a0a5c1a21db6114936f62190249a61da1f68b233 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 25 Oct 2021 12:06:53 +0300 Subject: [PATCH 064/173] Fix argument types for now and now64 --- src/Functions/now.cpp | 11 +++++++---- src/Functions/now64.cpp | 14 ++++++++++---- .../0_stateless/02100_now64_types_bug.reference | 3 +++ .../queries/0_stateless/02100_now64_types_bug.sql | 8 ++++++++ 4 files changed, 28 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02100_now64_types_bug.reference create mode 100644 tests/queries/0_stateless/02100_now64_types_bug.sql diff --git a/src/Functions/now.cpp b/src/Functions/now.cpp index b244d054cfe..653a39b38d0 100644 --- a/src/Functions/now.cpp +++ b/src/Functions/now.cpp @@ -43,13 +43,13 @@ private: class FunctionBaseNow : public IFunctionBase { public: - explicit FunctionBaseNow(time_t time_, DataTypePtr return_type_) : time_value(time_), return_type(return_type_) {} + explicit FunctionBaseNow(time_t time_, DataTypes argument_types_, DataTypePtr return_type_) + : time_value(time_), argument_types(std::move(argument_types_)), return_type(std::move(return_type_)) {} String getName() const override { return "now"; } const DataTypes & getArgumentTypes() const override { - static const DataTypes argument_types; return argument_types; } @@ -69,6 +69,7 @@ public: private: time_t time_value; + DataTypes argument_types; DataTypePtr return_type; }; @@ -117,8 +118,10 @@ public: } if (arguments.size() == 1) return std::make_unique( - time(nullptr), std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 0, 0))); - return std::make_unique(time(nullptr), std::make_shared()); + time(nullptr), DataTypes{arguments.front().type}, + std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 0, 0))); + + return std::make_unique(time(nullptr), DataTypes(), std::make_shared()); } }; diff --git a/src/Functions/now64.cpp b/src/Functions/now64.cpp index 4a3f4dbfb22..bd1038b1fc6 100644 --- a/src/Functions/now64.cpp +++ b/src/Functions/now64.cpp @@ -67,13 +67,13 @@ private: class FunctionBaseNow64 : public IFunctionBase { public: - explicit FunctionBaseNow64(Field time_, DataTypePtr return_type_) : time_value(time_), return_type(return_type_) {} + explicit FunctionBaseNow64(Field time_, DataTypes argument_types_, DataTypePtr return_type_) + : time_value(time_), argument_types(std::move(argument_types_)), return_type(std::move(return_type_)) {} String getName() const override { return "now64"; } const DataTypes & getArgumentTypes() const override { - static const DataTypes argument_types; return argument_types; } @@ -93,6 +93,7 @@ public: private: Field time_value; + DataTypes argument_types; DataTypePtr return_type; }; @@ -139,14 +140,19 @@ public: return std::make_shared(scale, timezone_name); } - FunctionBasePtr buildImpl(const ColumnsWithTypeAndName &, const DataTypePtr & result_type) const override + FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override { UInt32 scale = DataTypeDateTime64::default_scale; auto res_type = removeNullable(result_type); if (const auto * type = typeid_cast(res_type.get())) scale = type->getScale(); - return std::make_unique(nowSubsecond(scale), result_type); + DataTypes arg_types; + arg_types.reserve(arguments.size()); + for (const auto & arg : arguments) + arg_types.push_back(arg.type); + + return std::make_unique(nowSubsecond(scale), std::move(arg_types), std::move(result_type)); } }; diff --git a/tests/queries/0_stateless/02100_now64_types_bug.reference b/tests/queries/0_stateless/02100_now64_types_bug.reference new file mode 100644 index 00000000000..ae136939b64 --- /dev/null +++ b/tests/queries/0_stateless/02100_now64_types_bug.reference @@ -0,0 +1,3 @@ +2 +1 +0 diff --git a/tests/queries/0_stateless/02100_now64_types_bug.sql b/tests/queries/0_stateless/02100_now64_types_bug.sql new file mode 100644 index 00000000000..c91defcbf3a --- /dev/null +++ b/tests/queries/0_stateless/02100_now64_types_bug.sql @@ -0,0 +1,8 @@ +SELECT x +FROM +( + SELECT if((number % NULL) = -2147483648, NULL, if(toInt64(toInt64(now64(if((number % NULL) = -2147483648, NULL, if(toInt64(now64(toInt64(9223372036854775807, now64(h3kRing(NULL, NULL))), h3kRing(NULL, NULL))) = (number % NULL), nan, toFloat64(number))), toInt64(9223372036854775807, toInt64(9223372036854775807, now64(h3kRing(NULL, NULL))), now64(h3kRing(NULL, NULL))), h3kRing(NULL, NULL))), now64(toInt64(9223372036854775807, toInt64(0, now64(h3kRing(NULL, NULL))), now64(h3kRing(NULL, NULL))), h3kRing(NULL, NULL))) = (number % NULL), nan, toFloat64(number))) AS x + FROM system.numbers + LIMIT 3 +) +ORDER BY x DESC NULLS LAST From 2c20fbe5715c78effc9c8ba6989d6c3dfcf8740d Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 25 Oct 2021 12:46:50 +0300 Subject: [PATCH 065/173] Add ru translation for UTF normalized forms. --- .../functions/string-functions.md | 26 ++-- .../functions/string-functions.md | 144 ++++++++++++++++++ 2 files changed, 156 insertions(+), 14 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index f6dbadb875a..751b726765b 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -835,14 +835,14 @@ Type: [String](../../sql-reference/data-types/string.md). Query: ``` sql -SELECT length('ё'), normalizeUTF8NFC('ё') AS nfc, length(nfc) AS nfc_len; +SELECT length('â'), normalizeUTF8NFC('â') AS nfc, length(nfc) AS nfc_len; ``` Result: ``` text -┌─length('ё')─┬─nfc─┬─nfc_len─┐ -│ 2 │ ё │ 2 │ +┌─length('â')─┬─nfc─┬─nfc_len─┐ +│ 2 │ â │ 2 │ └─────────────┴─────┴─────────┘ ``` @@ -871,14 +871,14 @@ Type: [String](../../sql-reference/data-types/string.md). Query: ``` sql -SELECT length('ё'), normalizeUTF8NFD('ё') AS nfd, length(nfd) AS nfd_len; +SELECT length('â'), normalizeUTF8NFD('â') AS nfd, length(nfd) AS nfd_len; ``` Result: ``` text -┌─length('ё')─┬─nfd─┬─nfd_len─┐ -│ 2 │ ё │ 4 │ +┌─length('â')─┬─nfd─┬─nfd_len─┐ +│ 2 │ â │ 3 │ └─────────────┴─────┴─────────┘ ``` @@ -907,21 +907,19 @@ Type: [String](../../sql-reference/data-types/string.md). Query: ``` sql -SELECT length('ё'), normalizeUTF8NFKC('ё') AS nfkc, length(nfkc) AS nfkc_len; +SELECT length('â'), normalizeUTF8NFKC('â') AS nfkc, length(nfkc) AS nfkc_len; ``` Result: ``` text -┌─length('ё')─┬─nfkc─┬─nfkc_len─┐ -│ 2 │ ё │ 2 │ +┌─length('â')─┬─nfkc─┬─nfkc_len─┐ +│ 2 │ â │ 2 │ └─────────────┴──────┴──────────┘ - ``` ## normalizeUTF8NFKD {#normalizeutf8nfkd} - Converts a string to [NFKD normalized form](https://en.wikipedia.org/wiki/Unicode_equivalence#Normal_forms), assuming the string contains a set of bytes that make up a UTF-8 encoded text. **Syntax** @@ -945,14 +943,14 @@ Type: [String](../../sql-reference/data-types/string.md). Query: ``` sql -SELECT length('ё'), normalizeUTF8NFKD('ё') AS nfkd, length(nfkd) AS nfkd_len; +SELECT length('â'), normalizeUTF8NFKD('â') AS nfkd, length(nfkd) AS nfkd_len; ``` Result: ``` text -┌─length('ё')─┬─nfkd─┬─nfkd_len─┐ -│ 2 │ ё │ 4 │ +┌─length('â')─┬─nfkd─┬─nfkd_len─┐ +│ 2 │ â │ 3 │ └─────────────┴──────┴──────────┘ ``` diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 95e265f0b54..f3c627541ce 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -781,6 +781,150 @@ SELECT normalizedQueryHash('SELECT 1 AS `xyz`') != normalizedQueryHash('SELECT 1 └─────┘ ``` +## normalizeUTF8NFC {#normalizeutf8nfc} + +Преобразует строку в нормализованную форму [NFC](https://ru.wikipedia.org/wiki/Юникод#Алгоритмы_нормализации), предполагая, что строка содержит набор байтов, составляющих текст в кодировке UTF-8. + +**Синтаксис** + +``` sql +normalizeUTF8NFKD(words) +``` + +**Аргументы** + +- `words` — входная строка, которая содержит набор байтов, составляющих текст в кодировке UTF-8. [String](../../sql-reference/data-types/string.md). + +**Возвращаемое значение** + +- Строка, преобразованная в нормализированную форму NFC. + +Тип: [String](../../sql-reference/data-types/string.md). + +**Пример** + +Запрос: + +``` sql +SELECT length('â'), normalizeUTF8NFC('â') AS nfc, length(nfc) AS nfc_len; +``` + +Результат: + +``` text +┌─length('â')─┬─nfc─┬─nfc_len─┐ +│ 2 │ â │ 2 │ +└─────────────┴─────┴─────────┘ +``` + +## normalizeUTF8NFD {#normalizeutf8nfd} + +Преобразует строку в нормализованную форму [NFD](https://ru.wikipedia.org/wiki/Юникод#Алгоритмы_нормализации), предполагая, что строка содержит набор байтов, составляющих текст в кодировке UTF-8. + +**Синтаксис** + +``` sql +normalizeUTF8NFD(words) +``` + +**Аргументы** + +- `words` — входная строка, которая содержит набор байтов, составляющих текст в кодировке UTF-8. [String](../../sql-reference/data-types/string.md). + +**Возвращаемое значение** + +- Строка, преобразованная в нормализированную форму NFD. + +Тип: [String](../../sql-reference/data-types/string.md). + +**Пример** + +Запрос: + +``` sql +SELECT length('â'), normalizeUTF8NFD('â') AS nfd, length(nfd) AS nfd_len; +``` + +Результат: + +``` text +┌─length('â')─┬─nfd─┬─nfd_len─┐ +│ 2 │ â │ 3 │ +└─────────────┴─────┴─────────┘ +``` + +## normalizeUTF8NFKC {#normalizeutf8nfkc} + +Преобразует строку в нормализованную форму [NFKC](https://ru.wikipedia.org/wiki/Юникод#Алгоритмы_нормализации), предполагая, что строка содержит набор байтов, составляющих текст в кодировке UTF-8. + +**Синтаксис** + +``` sql +normalizeUTF8NFKC(words) +``` + +**Аргументы** + +- `words` — входная строка, которая содержит набор байтов, составляющих текст в кодировке UTF-8. [String](../../sql-reference/data-types/string.md). + +**Возвращаемое значение** + +- Строка, преобразованная в нормализированную форму NFKC. + +Тип: [String](../../sql-reference/data-types/string.md). + +**Пример** + +Запрос: + +``` sql +SELECT length('â'), normalizeUTF8NFKC('â') AS nfkc, length(nfkc) AS nfkc_len; +``` + +Результат: + +``` text +┌─length('â')─┬─nfkc─┬─nfkc_len─┐ +│ 2 │ â │ 2 │ +└─────────────┴──────┴──────────┘ +``` + +## normalizeUTF8NFKD {#normalizeutf8nfkd} + +Преобразует строку в нормализованную форму [NFKD](https://ru.wikipedia.org/wiki/Юникод#Алгоритмы_нормализации), предполагая, что строка содержит набор байтов, составляющих текст в кодировке UTF-8. + +**Синтаксис** + +``` sql +normalizeUTF8NFKD(words) +``` + +**Аргументы** + +- `words` — входная строка, которая содержит набор байтов, составляющих текст в кодировке UTF-8. [String](../../sql-reference/data-types/string.md). + +**Возвращаемое значение** + +- Строка, преобразованная в нормализированную форму NFKD. + +Тип: [String](../../sql-reference/data-types/string.md). + +**Пример** + +Запрос: + +``` sql +SELECT length('â'), normalizeUTF8NFKD('â') AS nfkd, length(nfkd) AS nfkd_len; +``` + +Результат: + +``` text +┌─length('â')─┬─nfkd─┬─nfkd_len─┐ +│ 2 │ â │ 3 │ +└─────────────┴──────┴──────────┘ +``` + ## encodeXMLComponent {#encode-xml-component} Экранирует символы для размещения строки в текстовом узле или атрибуте XML. From 264e173685c4abddfc1aa321eda5bbd1d36f9b9e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 25 Oct 2021 13:04:16 +0300 Subject: [PATCH 066/173] Update 02100_now64_types_bug.sql --- tests/queries/0_stateless/02100_now64_types_bug.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02100_now64_types_bug.sql b/tests/queries/0_stateless/02100_now64_types_bug.sql index c91defcbf3a..ef91b19bb08 100644 --- a/tests/queries/0_stateless/02100_now64_types_bug.sql +++ b/tests/queries/0_stateless/02100_now64_types_bug.sql @@ -1,7 +1,7 @@ SELECT x FROM ( - SELECT if((number % NULL) = -2147483648, NULL, if(toInt64(toInt64(now64(if((number % NULL) = -2147483648, NULL, if(toInt64(now64(toInt64(9223372036854775807, now64(h3kRing(NULL, NULL))), h3kRing(NULL, NULL))) = (number % NULL), nan, toFloat64(number))), toInt64(9223372036854775807, toInt64(9223372036854775807, now64(h3kRing(NULL, NULL))), now64(h3kRing(NULL, NULL))), h3kRing(NULL, NULL))), now64(toInt64(9223372036854775807, toInt64(0, now64(h3kRing(NULL, NULL))), now64(h3kRing(NULL, NULL))), h3kRing(NULL, NULL))) = (number % NULL), nan, toFloat64(number))) AS x + SELECT if((number % NULL) = -2147483648, NULL, if(toInt64(toInt64(now64(if((number % NULL) = -2147483648, NULL, if(toInt64(now64(toInt64(9223372036854775807, now64(plus(NULL, NULL))), plus(NULL, NULL))) = (number % NULL), nan, toFloat64(number))), toInt64(9223372036854775807, toInt64(9223372036854775807, now64(plus(NULL, NULL))), now64(plus(NULL, NULL))), plus(NULL, NULL))), now64(toInt64(9223372036854775807, toInt64(0, now64(plus(NULL, NULL))), now64(plus(NULL, NULL))), plus(NULL, NULL))) = (number % NULL), nan, toFloat64(number))) AS x FROM system.numbers LIMIT 3 ) From 538d71b855aa1c2e36c586eab838d409ba2ffbe4 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 25 Oct 2021 13:44:46 +0300 Subject: [PATCH 067/173] Update docs/ru/sql-reference/functions/string-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index f3c627541ce..10ecc6c0a83 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -788,7 +788,7 @@ SELECT normalizedQueryHash('SELECT 1 AS `xyz`') != normalizedQueryHash('SELECT 1 **Синтаксис** ``` sql -normalizeUTF8NFKD(words) +normalizeUTF8NFC(words) ``` **Аргументы** From a211e48c3c1e910d2ecac1327f51cd368ddf0a9f Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 25 Oct 2021 13:44:50 +0300 Subject: [PATCH 068/173] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 751b726765b..4391d013440 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -817,7 +817,7 @@ Converts a string to [NFC normalized form](https://en.wikipedia.org/wiki/Unicode **Syntax** ``` sql -normalizeUTF8NFKD(words) +normalizeUTF8NFC(words) ``` **Arguments** From 8ec5d68ce3d3ef1ca023db32618daf2acecf1239 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Mon, 25 Oct 2021 15:02:24 +0300 Subject: [PATCH 069/173] Apply suggestions from code review Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/operations/settings/settings.md | 6 +++--- docs/en/sql-reference/statements/insert-into.md | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f02fe383f37..3eb39907991 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3817,7 +3817,7 @@ Default value: `0`. ## async_insert {#async-insert} -Enables or disables asynchronous inserts for multiple concurrent connetions. This makes sense only for insertion over HTTP protocol. Note, that deduplications isn't working for such inserts. +Enables or disables asynchronous inserts for multiple concurrent connections. This makes sense only for insertion over HTTP protocol. Note that deduplication isn't working for such inserts. If enabled, the data is combined into batches before the insertion into tables, so it is possible to do small and frequent insertions into ClickHouse (up to 15000 queries per second) without buffer tables. @@ -3827,8 +3827,8 @@ If [wait_for_async_insert](#wait-for-async-insert) is enabled, every client will Possible values: -- 0 — Disabled. -- 1 — Enabled. +- 0 — Insertions are made synchronously, one after another. +- 1 — Multiple asynchronous insertions enabled. Default value: `0`. diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index 83812a198d9..c44f7435089 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -119,4 +119,4 @@ Performance will not decrease if: - Data is added in real time. - You upload data that is usually sorted by time. -It's also possible to asynchronously insert data in small but frequent inserts from multiple concurrent connections with the setting [async_insert](../../operations/settings/settings.md#async-insert). The data from such inserts is combined into batches and then safely inserted into a table. +It's also possible to asynchronously insert data in small but frequent inserts from multiple concurrent connections. The data from such insertions is combined into batches and then safely inserted into a table. To enable the asynchronous mode, switch on the [async_insert](../../operations/settings/settings.md#async-insert) setting. Note that asynchronous insertions are supported only over HTTP protocol, and deduplication is not supported for them. From 0642aeca3a9e71b74912ca3e58533f1c724aa57d Mon Sep 17 00:00:00 2001 From: gyuton Date: Mon, 25 Oct 2021 15:55:04 +0300 Subject: [PATCH 070/173] Revert "Merge branch 'gyuton-DOCSUP-16593-document-tuple-functions-and-operators' of https://github.com/gyuton/ClickHouse into gyuton-DOCSUP-14941-Document-asynchronous-insert-mode" This reverts commit ccad91b5effbc3d1d9d5007b01e6a011ac950dab, reversing changes made to 4ecd2ed6e204d4d70200810c43d5370dcbd61ee9. --- .../functions/arithmetic-functions.md | 73 -- .../functions/tuple-functions.md | 801 ----------------- docs/en/sql-reference/operators/index.md | 10 - .../functions/arithmetic-functions.md | 75 -- .../functions/tuple-functions.md | 802 ------------------ docs/ru/sql-reference/operators/index.md | 10 - 6 files changed, 1771 deletions(-) diff --git a/docs/en/sql-reference/functions/arithmetic-functions.md b/docs/en/sql-reference/functions/arithmetic-functions.md index 40fadf34eab..3187f13b5b9 100644 --- a/docs/en/sql-reference/functions/arithmetic-functions.md +++ b/docs/en/sql-reference/functions/arithmetic-functions.md @@ -82,76 +82,3 @@ An exception is thrown when dividing by zero or when dividing a minimal negative Returns the least common multiple of the numbers. An exception is thrown when dividing by zero or when dividing a minimal negative number by minus one. -## max2 {#max2} - -Compares two values and returns the maximum. The returned value is converted to [Float64](../../sql-reference/data-types/float.md). - -**Syntax** - -```sql -max2(value1, value2) -``` - -**Arguments** - -- `value1` — First value. [Int/UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). -- `value2` — Second value. [Int/UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). - -**Returned value** - -- The maximum of two values. - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT max2(-1, 2); -``` - -Result: - -```text -┌─max2(-1, 2)─┐ -│ 2 │ -└─────────────┘ -``` - -## min2 {#min2} - -Compares two values and returns the minimum. The returned value is converted to [Float64](../../sql-reference/data-types/float.md). - -**Syntax** - -```sql -max2(value1, value2) -``` - -**Arguments** - -- `value1` — First value. [Int/UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). -- `value2` — Second value. [Int/UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). - -**Returned value** - -- The minimum of two values. - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT min2(-1, 2); -``` - -Result: - -```text -┌─min2(-1, 2)─┐ -│ -1 │ -└─────────────┘ -``` diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 73ebd9ad7d6..39e59ae2ba9 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -165,804 +165,3 @@ Result: │ 2 │ └─────────────────┘ ``` - -## tuplePlus {#tupleplus} - -Calculates the sum of corresponding values of two tuples of the same size. - -**Syntax** - -```sql -tuplePlus(tuple1, tuple2) -``` - -Alias: `vectorSum`. - -**Arguments** - -- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Tuple with the sum. - -Type: [Tuple](../../sql-reference/data-types/tuple.md). - -**Example** - -Query: - -```sql -SELECT tuplePlus((1, 2), (2, 3)); -``` - -Result: - -```text -┌─tuplePlus((1, 2), (2, 3))─┐ -│ (3,5) │ -└───────────────────────────┘ -``` - -## tupleMinus {#tupleminus} - -Calculates the subtraction of corresponding values of two tuples of the same size. - -**Syntax** - -```sql -tupleMinus(tuple1, tuple2) -``` - -Alias: `vectorDifference`. - -**Arguments** - -- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Tuple with the result of subtraction. - -Type: [Tuple](../../sql-reference/data-types/tuple.md). - -**Example** - -Query: - -```sql -SELECT tupleMinus((1, 2), (2, 3)); -``` - -Result: - -```text -┌─tupleMinus((1, 2), (2, 3))─┐ -│ (-1,-1) │ -└────────────────────────────┘ -``` - -## tupleMultiply {#tuplemultiply} - -Calculates the multiplication of corresponding values of two tuples of the same size. - -**Syntax** - -```sql -tupleMultiply(tuple1, tuple2) -``` - -**Arguments** - -- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Tuple with the multiplication. - -Type: [Tuple](../../sql-reference/data-types/tuple.md). - -**Example** - -Query: - -```sql -SELECT tupleMultiply((1, 2), (2, 3)); -``` - -Result: - -```text -┌─tupleMultiply((1, 2), (2, 3))─┐ -│ (2,6) │ -└───────────────────────────────┘ -``` - -## tupleDivide {#tupledivide} - -Calculates the division of corresponding values of two tuples of the same size. Note that division by zero will return `inf`. - -**Syntax** - -```sql -tupleDivide(tuple1, tuple2) -``` - -**Arguments** - -- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Tuple with the result of division. - -Type: [Tuple](../../sql-reference/data-types/tuple.md). - -**Example** - -Query: - -```sql -SELECT tupleDivide((1, 2), (2, 3)); -``` - -Result: - -```text -┌─tupleDivide((1, 2), (2, 3))─┐ -│ (0.5,0.6666666666666666) │ -└─────────────────────────────┘ -``` - -## tupleNegate {#tuplenegate} - -Calculates the negation of the tuple values. - -**Syntax** - -```sql -tupleNegate(tuple) -``` - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Tuple with the result of negation. - -Type: [Tuple](../../sql-reference/data-types/tuple.md). - -**Example** - -Query: - -```sql -SELECT tupleDivide((1, 2)); -``` - -Result: - -```text -┌─tupleNegate((1, 2))─┐ -│ (-1,-2) │ -└─────────────────────┘ -``` - -## tupleMultiplyByNumber {#tuplemultiplybynumber} - -Returns a tuple with all values multiplied by a number. - -**Syntax** - -```sql -tupleMultiplyByNumber(tuple, number) -``` - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). -- `number` — Multiplier. [Int/UInt](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). - -**Returned value** - -- Tuple with multiplied values. - -Type: [Tuple](../../sql-reference/data-types/tuple.md). - -**Example** - -Query: - -```sql -SELECT tupleMultiplyByNumber((1, 2), -2.1); -``` - -Result: - -```text -┌─tupleMultiplyByNumber((1, 2), -2.1)─┐ -│ (-2.1,-4.2) │ -└─────────────────────────────────────┘ -``` - -## tupleDivideByNumber {#tupledividebynumber} - -Returns a tuple with all values divided by a number. Note that division by zero will return `inf`. - -**Syntax** - -```sql -tupleDivideByNumber(tuple, number) -``` - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). -- `number` — Divider. [Int/UInt](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). - -**Returned value** - -- Tuple with divided values. - -Type: [Tuple](../../sql-reference/data-types/tuple.md). - -**Example** - -Query: - -```sql -SELECT tupleDivideByNumber((1, 2), 0.5); -``` - -Result: - -```text -┌─tupleDivideByNumber((1, 2), 0.5)─┐ -│ (2,4) │ -└──────────────────────────────────┘ -``` - -## dotProduct {#dotproduct} - -Calculates the scalar product of two tuples of the same size. - -**Syntax** - -```sql -dotProduct(tuple1, tuple2) -``` - -Alias: `scalarProduct`. - -**Arguments** - -- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Scalar product. - -Type: [Int/UInt](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). - -**Example** - -Query: - -```sql -SELECT dotProduct((1, 2), (2, 3)); -``` - -Result: - -```text -┌─dotProduct((1, 2), (2, 3))─┐ -│ 8 │ -└────────────────────────────┘ -``` - -## L1Norm {#l1norm} - -Calculates the sum of absolute values of a tuple. - -**Syntax** - -```sql -L1Norm(tuple) -``` - -Alias: `normL1`. - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- L1-norm or [taxicab geometry](https://en.wikipedia.org/wiki/Taxicab_geometry) distance. - -Type: [UInt](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). - -**Example** - -Query: - -```sql -SELECT L1Norm((1, 2)); -``` - -Result: - -```text -┌─L1Norm((1, 2))─┐ -│ 3 │ -└────────────────┘ -``` - -## L2Norm {#l2norm} - -Calculates the square root of the sum of the squares of the tuple values. - -**Syntax** - -```sql -L2Norm(tuple) -``` - -Alias: `normL2`. - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- L2-norm or [Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance). - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT L2Norm((1, 2)); -``` - -Result: - -```text -┌───L2Norm((1, 2))─┐ -│ 2.23606797749979 │ -└──────────────────┘ -``` - -## LinfNorm {#linfnorm} - -Calculates the maximum of absolute values of a tuple. - -**Syntax** - -```sql -LinfNorm(tuple) -``` - -Alias: `normLinf`. - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Linf-norm or the maximum absolute value. - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT LinfNorm((1, -2)); -``` - -Result: - -```text -┌─LinfNorm((1, -2))─┐ -│ 2 │ -└───────────────────┘ -``` - -## LpNorm {#lpnorm} - -Calculates the root of `p`-th power of the sum of the absolute values of a tuple in the power of `p`. - -**Syntax** - -```sql -LpNorm(tuple, p) -``` - -Alias: `normLp`. - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). -- `p` — The power. Possible values: any number from [1;inf). [UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). - -**Returned value** - -- [Lp-norm](https://en.wikipedia.org/wiki/Norm_(mathematics)#p-norm) - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT LpNorm((1, -2),2); -``` - -Result: - -```text -┌─LpNorm((1, -2), 2)─┐ -│ 2.23606797749979 │ -└────────────────────┘ -``` - -## L1Distance {#l1distance} - -Calculates the distance between two points (the values of the tuples are the coordinates) in 1-norm ([taxicab geometry](https://en.wikipedia.org/wiki/Taxicab_geometry) distance). - -**Syntax** - -```sql -L1Distance(tuple1, tuple2) -``` - -Alias: `distanceL1`. - -**Arguments** - -- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple1` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- 1-norm distance. - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT L1Distance((1, 2), (2, 3)); -``` - -Result: - -```text -┌─L1Distance((1, 2), (2, 3))─┐ -│ 2 │ -└────────────────────────────┘ -``` - -## L2Distance {#l2distance} - -Calculates the distance between two points (the values of the tuples are the coordinates) in 2-norm ([Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance)). - -**Syntax** - -```sql -L2Distance(tuple1, tuple2) -``` - -Alias: `distanceL2`. - -**Arguments** - -- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple1` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- 2-norm distance. - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT L2Distance((1, 2), (2, 3)); -``` - -Result: - -```text -┌─L2Distance((1, 2), (2, 3))─┐ -│ 1.4142135623730951 │ -└────────────────────────────┘ -``` - -## LinfDistance {#linfdistance} - -Calculates the distance between two points (the values of the tuples are the coordinates) in [infinity-norm distance](https://en.wikipedia.org/wiki/Norm_(mathematics)#Maximum_norm_(special_case_of:_infinity_norm,_uniform_norm,_or_supremum_norm)). - -**Syntax** - -```sql -LinfDistance(tuple1, tuple2) -``` - -Alias: `distanceLinf`. - -**Arguments** - -- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple1` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Infinity-norm distance. - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT LinfDistance((1, 2), (2, 3)); -``` - -Result: - -```text -┌─LinfDistance((1, 2), (2, 3))─┐ -│ 1 │ -└──────────────────────────────┘ -``` - -## LpDistance {#lpdistance} - -Calculates the distance between two points (the values of the tuples are the coordinates) in [p-norm distance](https://en.wikipedia.org/wiki/Norm_(mathematics)#p-norm). - -**Syntax** - -```sql -LpDistance(tuple1, tuple2, p) -``` - -Alias: `distanceLp`. - -**Arguments** - -- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple1` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `p` — The power. Possible values: any number from [1;inf). [UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). - -**Returned value** - -- p-norm distance. - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT LpDistance((1, 2), (2, 3), 3); -``` - -Result: - -```text -┌─LpDistance((1, 2), (2, 3), 3)─┐ -│ 1.2599210498948732 │ -└───────────────────────────────┘ -``` - -## L1Normalize {#l1normalize} - -Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in 1-norm ([taxicab geometry](https://en.wikipedia.org/wiki/Taxicab_geometry)). - -**Syntax** - -```sql -L1Normalize(tuple) -``` - -Alias: `normalizeL1`. - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Unit vector. - -Type: [Tuple](../../sql-reference/data-types/tuple.md) of [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT L1Normalize((1, 2)); -``` - -Result: - -```text -┌─L1Normalize((1, 2))─────────────────────┐ -│ (0.3333333333333333,0.6666666666666666) │ -└─────────────────────────────────────────┘ -``` - -## L2Normalize {#l2normalize} - -Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in 2-norm ([Euclidean geometry](https://en.wikipedia.org/wiki/Euclidean_space)). - -**Syntax** - -```sql -L2Normalize(tuple) -``` - -Alias: `normalizeL1`. - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Unit vector. - -Type: [Tuple](../../sql-reference/data-types/tuple.md) of [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT L2Normalize((3, 4)); -``` - -Result: - -```text -┌─L2Normalize((3, 4))─┐ -│ (0.6,0.8) │ -└─────────────────────┘ -``` - -## LinfNormalize {#linfnormalize} - -Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in [infinity-norm](https://en.wikipedia.org/wiki/Norm_(mathematics)#Maximum_norm_(special_case_of:_infinity_norm,_uniform_norm,_or_supremum_norm)). - -**Syntax** - -```sql -LinfNormalize(tuple) -``` - -Alias: `normalizeLinf `. - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Unit vector. - -Type: [Tuple](../../sql-reference/data-types/tuple.md) of [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT LinfNormalize((3, 4)); -``` - -Result: - -```text -┌─LinfNormalize((3, 4))─┐ -│ (0.75,1) │ -└───────────────────────┘ -``` - -## LpNormalize {#lpnormalize} - -Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in [p-norm](https://en.wikipedia.org/wiki/Norm_(mathematics)#p-norm). - -**Syntax** - -```sql -LpNormalize(tuple, p) -``` - -Alias: `normalizeLp `. - -**Arguments** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). -- `p` — The power. Possible values: any number from [1;inf). [UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). - -**Returned value** - -- Unit vector. - -Type: [Tuple](../../sql-reference/data-types/tuple.md) of [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT LpNormalize((3, 4),5); -``` - -Result: - -```text -┌─LpNormalize((3, 4), 5)──────────────────┐ -│ (0.7187302630182624,0.9583070173576831) │ -└─────────────────────────────────────────┘ -``` - -## cosineDistance {#cosinedistance} - -Calculates the cosine distance between two vectors (the values of the tuples are the coordinates). The less the returned value is, the more similar are the vectors. - -**Syntax** - -```sql -cosineDistance(tuple1, tuple2) -``` - -**Arguments** - -- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). - -**Returned value** - -- Cosine of the angle between two vectors substracted from one. - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT cosineDistance((1, 2),(2,3)); -``` - -Result: - -```text -┌─cosineDistance((1, 2), (2, 3))─┐ -│ 0.007722123286332261 │ -└────────────────────────────────┘ -``` diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index aa9b7f3ba63..dbda5f5dd69 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -17,30 +17,20 @@ ClickHouse transforms operators to their corresponding functions at the query pa `-a` – The `negate (a)` function. -For tuple negation: [tupleNegate](../../sql-reference/functions/tuple-functions.md#tuplenegate). - ## Multiplication and Division Operators {#multiplication-and-division-operators} `a * b` – The `multiply (a, b)` function. -For multiplying tuple by number: [tupleMultiplyByNumber](../../sql-reference/functions/tuple-functions.md#tuplemultiplybynumber), for scalar profuct: [dotProduct](../../sql-reference/functions/tuple-functions.md#dotproduct). - `a / b` – The `divide(a, b)` function. -For dividing tuple by number: [tupleDivideByNumber](../../sql-reference/functions/tuple-functions.md#tupledividebynumber). - `a % b` – The `modulo(a, b)` function. ## Addition and Subtraction Operators {#addition-and-subtraction-operators} `a + b` – The `plus(a, b)` function. -For tuple addiction: [tuplePlus](../../sql-reference/functions/tuple-functions.md#tupleplus). - `a - b` – The `minus(a, b)` function. -For tuple subtraction: [tupleMinus](../../sql-reference/functions/tuple-functions.md#tupleminus). - ## Comparison Operators {#comparison-operators} `a = b` – The `equals(a, b)` function. diff --git a/docs/ru/sql-reference/functions/arithmetic-functions.md b/docs/ru/sql-reference/functions/arithmetic-functions.md index 278f3edb74a..f587b7b5b5d 100644 --- a/docs/ru/sql-reference/functions/arithmetic-functions.md +++ b/docs/ru/sql-reference/functions/arithmetic-functions.md @@ -83,78 +83,3 @@ SELECT toTypeName(0), toTypeName(0 + 0), toTypeName(0 + 0 + 0), toTypeName(0 + 0 Вычисляет наименьшее общее кратное чисел. При делении на ноль или при делении минимального отрицательного числа на минус единицу, кидается исключение. - -## max2 {#max2} - -Сравнивает два числа и возвращает максимум. Возвращаемое значение приводится к типу [Float64](../../sql-reference/data-types/float.md). - -**Синтаксис** - -```sql -max2(value1, value2) -``` - -**Аргументы** - -- `value1` — первое число. [Int/UInt](../../sql-reference/data-types/int-uint.md) или [Float](../../sql-reference/data-types/float.md). -- `value2` — второе число. [Int/UInt](../../sql-reference/data-types/int-uint.md) или [Float](../../sql-reference/data-types/float.md). - -**Возвращаемое значение** - -- Максимальное значение среди двух чисел. - -Тип: [Float](../../sql-reference/data-types/float.md). - -**Пример** - -Запрос: - -```sql -SELECT max2(-1, 2); -``` - -Результат: - -```text -┌─max2(-1, 2)─┐ -│ 2 │ -└─────────────┘ -``` - -## min2 {#min2} - -Сравнивает два числа и возвращает минимум. Возвращаемое значение приводится к типу [Float64](../../sql-reference/data-types/float.md). - -**Синтаксис** - -```sql -min2(value1, value2) -``` - -**Аргументы** - -- `value1` — первое число. [Int/UInt](../../sql-reference/data-types/int-uint.md) или [Float](../../sql-reference/data-types/float.md). -- `value2` — второе число. [Int/UInt](../../sql-reference/data-types/int-uint.md) или [Float](../../sql-reference/data-types/float.md). - -**Возвращаемое значение** - -- Минимальное значение среди двух чисел. - -Тип: [Float](../../sql-reference/data-types/float.md). - -**Пример** - -Запрос: - -```sql -SELECT min2(-1, 2); -``` - -Результат: - -```text -┌─min2(-1, 2)─┐ -│ -1 │ -└─────────────┘ -``` - diff --git a/docs/ru/sql-reference/functions/tuple-functions.md b/docs/ru/sql-reference/functions/tuple-functions.md index 9ee3f413577..381743a450b 100644 --- a/docs/ru/sql-reference/functions/tuple-functions.md +++ b/docs/ru/sql-reference/functions/tuple-functions.md @@ -163,805 +163,3 @@ SELECT tupleHammingDistance(wordShingleMinHash(string), wordShingleMinHashCaseIn │ 2 │ └─────────────────┘ ``` - -## tuplePlus {#tupleplus} - -Вычисляет сумму соответствующих значений двух кортежей одинакового размера. - -**Синтаксис** - -```sql -tuplePlus(tuple1, tuple2) -``` - -Синоним: `vectorSum`. - -**Аргументы** - -- `tuple1` — первый кортеж. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — второй кортеж. [Tuple](../../sql-reference/data-types/tuple.md). - -**Возвращаемое значение** - -- Кортеж с суммами. - -Тип: [Tuple](../../sql-reference/data-types/tuple.md). - -**Пример** - -Запрос: - -```sql -SELECT tuplePlus((1, 2), (2, 3)); -``` - -Результат: - -```text -┌─tuplePlus((1, 2), (2, 3))─┐ -│ (3,5) │ -└───────────────────────────┘ -``` - -## tupleMinus {#tupleminus} - -Вычисляет разность соответствующих значений двух кортежей одинакового размера. - -**Синтаксис** - -```sql -tupleMinus(tuple1, tuple2) -``` - -Синоним: `vectorDifference`. - -**Аргументы** - -- `tuple1` — первый кортеж. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — второй кортеж. [Tuple](../../sql-reference/data-types/tuple.md). - -**Возвращаемое значение** - -- Кортеж с разностями. - -Тип: [Tuple](../../sql-reference/data-types/tuple.md). - -**Пример** - -Запрос: - -```sql -SELECT tupleMinus((1, 2), (2, 3)); -``` - -Результат: - -```text -┌─tupleMinus((1, 2), (2, 3))─┐ -│ (-1,-1) │ -└────────────────────────────┘ -``` - -## tupleMultiply {#tuplemultiply} - -Вычисляет произведение соответствующих значений двух кортежей одинакового размера. - -**Синтаксис** - -```sql -tupleMultiply(tuple1, tuple2) -``` - -**Аргументы** - -- `tuple1` — первый кортеж. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — второй кортеж. [Tuple](../../sql-reference/data-types/tuple.md). - -**Возвращаемое значение** - -- Кортеж с произведениями. - -Тип: [Tuple](../../sql-reference/data-types/tuple.md). - -**Пример** - -Запрос: - -```sql -SELECT tupleMultiply((1, 2), (2, 3)); -``` - -Результат: - -```text -┌─tupleMultiply((1, 2), (2, 3))─┐ -│ (2,6) │ -└───────────────────────────────┘ -``` - -## tupleDivide {#tupledivide} - -Вычисляет частное соответствующих значений двух кортежей одинакового размера. Обратите внимание, что при делении на ноль возвращается значение `inf`. - -**Синтаксис** - -```sql -tupleDivide(tuple1, tuple2) -``` - -**Аргументы** - -- `tuple1` — первый кортеж. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — второй кортеж. [Tuple](../../sql-reference/data-types/tuple.md). - -**Возвращаемое значение** - -- Кортеж с частными. - -Тип: [Tuple](../../sql-reference/data-types/tuple.md). - -**Пример** - -Запрос: - -```sql -SELECT tupleDivide((1, 2), (2, 3)); -``` - -Результат: - -```text -┌─tupleDivide((1, 2), (2, 3))─┐ -│ (0.5,0.6666666666666666) │ -└─────────────────────────────┘ -``` - -## tupleNegate {#tuplenegate} - -Применяет отрицание ко всем значениям кортежа. - -**Синтаксис** - -```sql -tupleNegate(tuple) -``` - -**Аргументы** - -- `tuple` — кортеж. [Tuple](../../sql-reference/data-types/tuple.md). - -**Возвращаемое значение** - -- Кортеж с результатом отрицания. - -Тип: [Tuple](../../sql-reference/data-types/tuple.md). - -**Пример** - -Запрос: - -```sql -SELECT tupleDivide((1, 2)); -``` - -Результат: - -```text -┌─tupleNegate((1, 2))─┐ -│ (-1,-2) │ -└─────────────────────┘ -``` - -## tupleMultiplyByNumber {#tuplemultiplybynumber} - -Возвращает кортеж, в котором все значения умножены на заданное число. - -**Синтаксис** - -```sql -tupleMultiplyByNumber(tuple, number) -``` - -**Аргументы** - -- `tuple` — кортеж. [Tuple](../../sql-reference/data-types/tuple.md). -- `number` — множитель. [Int/UInt](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) или [Decimal](../../sql-reference/data-types/decimal.md). - -**Возвращаемое значение** - -- Кортеж с результатами умножения на число. - -Тип: [Tuple](../../sql-reference/data-types/tuple.md). - -**Пример** - -Запрос: - -```sql -SELECT tupleMultiplyByNumber((1, 2), -2.1); -``` - -Результат: - -```text -┌─tupleMultiplyByNumber((1, 2), -2.1)─┐ -│ (-2.1,-4.2) │ -└─────────────────────────────────────┘ -``` - -## tupleDivideByNumber {#tupledividebynumber} - -Возвращает кортеж, в котором все значения поделены на заданное число. Обратите внимание, что при делении на ноль возвращается значение `inf`. - -**Синтаксис** - -```sql -tupleDivideByNumber(tuple, number) -``` - -**Аргументы** - -- `tuple` — кортеж. [Tuple](../../sql-reference/data-types/tuple.md). -- `number` — делитель. [Int/UInt](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). - -**Возвращаемое значение** - -- Кортеж с результатами деления на число. - -Тип: [Tuple](../../sql-reference/data-types/tuple.md). - -**Пример** - -Запрос: - -```sql -SELECT tupleDivideByNumber((1, 2), 0.5); -``` - -Результат: - -```text -┌─tupleDivideByNumber((1, 2), 0.5)─┐ -│ (2,4) │ -└──────────────────────────────────┘ -``` - -## dotProduct {#dotproduct} - -Вычисляет скалярное произведение двух кортежей одинакового размера. - -**Синтаксис** - -```sql -dotProduct(tuple1, tuple2) -``` - -Синоним: `scalarProduct`. - -**Аргументы** - -- `tuple1` — первый кортеж. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — второй кортеж. [Tuple](../../sql-reference/data-types/tuple.md). - - -**Возвращаемое значение** - -- Скалярное произведение. - -Тип: [Int/UInt](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) или [Decimal](../../sql-reference/data-types/decimal.md). - -**Пример** - -Запрос: - -```sql -SELECT dotProduct((1, 2), (2, 3)); -``` - -Результат: - -```text -┌─dotProduct((1, 2), (2, 3))─┐ -│ 8 │ -└────────────────────────────┘ -``` - -## L1Norm {#l1norm} - -Вычисляет сумму абсолютных значений кортежа. - -**Синтаксис** - -```sql -L1Norm(tuple) -``` - -Синоним: `normL1`. - -**Аргументы** - -- `tuple` — кортеж. [Tuple](../../sql-reference/data-types/tuple.md). - -**Возвращаемое значение** - -- L1-норма или [расстояние городских кварталов](https://ru.wikipedia.org/wiki/%D0%A0%D0%B0%D1%81%D1%81%D1%82%D0%BE%D1%8F%D0%BD%D0%B8%D0%B5_%D0%B3%D0%BE%D1%80%D0%BE%D0%B4%D1%81%D0%BA%D0%B8%D1%85_%D0%BA%D0%B2%D0%B0%D1%80%D1%82%D0%B0%D0%BB%D0%BE%D0%B2). - -Тип: [UInt](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) или [Decimal](../../sql-reference/data-types/decimal.md). - -**Пример** - -Запрос: - -```sql -SELECT L1Norm((1, 2)); -``` - -Результат: - -```text -┌─L1Norm((1, 2))─┐ -│ 3 │ -└────────────────┘ -``` - -## L2Norm {#l2norm} - -Вычисляет квадратный корень из суммы квадратов значений кортежа. - -**Синтаксис** - -```sql -L2Norm(tuple) -``` - -Синоним: `normL2`. - -**Аргументы** - -- `tuple` — кортеж. [Tuple](../../sql-reference/data-types/tuple.md). - -**Возвращаемое значение** - -- L2-норма или [Евклидово расстояние](https://ru.wikipedia.org/wiki/%D0%95%D0%B2%D0%BA%D0%BB%D0%B8%D0%B4%D0%BE%D0%B2%D0%B0_%D0%BC%D0%B5%D1%82%D1%80%D0%B8%D0%BA%D0%B0). - -Тип: [Float](../../sql-reference/data-types/float.md). - -**Пример** - -Запрос: - -```sql -SELECT L2Norm((1, 2)); -``` - -Результат: - -```text -┌───L2Norm((1, 2))─┐ -│ 2.23606797749979 │ -└──────────────────┘ -``` - -## LinfNorm {#linfnorm} - -Вычисляет максимум из абсолютных значений кортежа. - -**Синтаксис** - -```sql -LinfNorm(tuple) -``` - -Синоним: `normLinf`. - -**Аргументы** - -- `tuple` — кортеж. [Tuple](../../sql-reference/data-types/tuple.md). - -**Возвращаемое значение** - -- Linf-норма или максимальное абсолютное значение. - -Тип: [Float](../../sql-reference/data-types/float.md). - -**Пример** - -Запрос: - -```sql -SELECT LinfNorm((1, -2)); -``` - -Результат: - -```text -┌─LinfNorm((1, -2))─┐ -│ 2 │ -└───────────────────┘ -``` - -## LpNorm {#lpnorm} - -Возвращает корень степени `p` из суммы абсолютных значений кортежа, возведенных в степень `p`. - -**Синтаксис** - -```sql -LpNorm(tuple, p) -``` - -Синоним: `normLp`. - -**Аргументы** - -- `tuple` — кортеж. [Tuple](../../sql-reference/data-types/tuple.md). -- `p` — степень. Возможные значение: любое число из промежутка [1;inf). [UInt](../../sql-reference/data-types/int-uint.md) или [Float](../../sql-reference/data-types/float.md). - -**Возвращаемое значение** - -- [Lp-норма](https://ru.wikipedia.org/wiki/%D0%9D%D0%BE%D1%80%D0%BC%D0%B0_(%D0%BC%D0%B0%D1%82%D0%B5%D0%BC%D0%B0%D1%82%D0%B8%D0%BA%D0%B0)#%D0%9D%D0%B5%D0%BA%D0%BE%D1%82%D0%BE%D1%80%D1%8B%D0%B5_%D0%B2%D0%B8%D0%B4%D1%8B_%D0%BC%D0%B0%D1%82%D1%80%D0%B8%D1%87%D0%BD%D1%8B%D1%85_%D0%BD%D0%BE%D1%80%D0%BC) - -Тип: [Float](../../sql-reference/data-types/float.md). - -**Пример** - -Запрос: - -```sql -SELECT LpNorm((1, -2),2); -``` - -Результат: - -```text -┌─LpNorm((1, -2), 2)─┐ -│ 2.23606797749979 │ -└────────────────────┘ -``` - -## L1Distance {#l1distance} - -Вычисляет расстояние между двумя точками (значения кортежей — координаты точек) в норме L1 ([расстояние городских кварталов](https://ru.wikipedia.org/wiki/%D0%A0%D0%B0%D1%81%D1%81%D1%82%D0%BE%D1%8F%D0%BD%D0%B8%D0%B5_%D0%B3%D0%BE%D1%80%D0%BE%D0%B4%D1%81%D0%BA%D0%B8%D1%85_%D0%BA%D0%B2%D0%B0%D1%80%D1%82%D0%B0%D0%BB%D0%BE%D0%B2)). - -**Синтаксис** - -```sql -L1Distance(tuple1, tuple2) -``` - -Синоним: `distanceL1`. - -**Аргументы** - -- `tuple1` — первый кортеж. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — второй кортеж. [Tuple](../../sql-reference/data-types/tuple.md). - -**Возвращаемое значение** - -- Расстояние в норме L1. - -Тип: [Float](../../sql-reference/data-types/float.md). - -**Пример** - -Запрос: - -```sql -SELECT L1Distance((1, 2), (2, 3)); -``` - -Результат: - -```text -┌─L1Distance((1, 2), (2, 3))─┐ -│ 2 │ -└────────────────────────────┘ -``` - -## L2Distance {#l2distance} - -Вычисляет расстояние между двумя точками (значения кортежей — координаты точек) в норме L2 ([Евклидово расстояние](https://ru.wikipedia.org/wiki/%D0%95%D0%B2%D0%BA%D0%BB%D0%B8%D0%B4%D0%BE%D0%B2%D0%B0_%D0%BC%D0%B5%D1%82%D1%80%D0%B8%D0%BA%D0%B0)). - -**Синтаксис** - -```sql -L2Distance(tuple1, tuple2) -``` - -Синоним: `distanceL2`. - -**Аргументы** - -- `tuple1` — первый кортеж. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — второй кортеж. [Tuple](../../sql-reference/data-types/tuple.md). - -**Возвращаемое значение** - -- Расстояние в норме L2. - -Тип: [Float](../../sql-reference/data-types/float.md). - -**Пример** - -Запрос: - -```sql -SELECT L2Distance((1, 2), (2, 3)); -``` - -Результат: - -```text -┌─L2Distance((1, 2), (2, 3))─┐ -│ 1.4142135623730951 │ -└────────────────────────────┘ -``` - -## LinfDistance {#linfdistance} - -Вычисляет расстояние между двумя точками (значения кортежей — координаты точек) в норме [Linf](https://ru.wikipedia.org/wiki/%D0%9D%D0%BE%D1%80%D0%BC%D0%B0_(%D0%BC%D0%B0%D1%82%D0%B5%D0%BC%D0%B0%D1%82%D0%B8%D0%BA%D0%B0)#%D0%9D%D0%B5%D0%BA%D0%BE%D1%82%D0%BE%D1%80%D1%8B%D0%B5_%D0%B2%D0%B8%D0%B4%D1%8B_%D0%BC%D0%B0%D1%82%D1%80%D0%B8%D1%87%D0%BD%D1%8B%D1%85_%D0%BD%D0%BE%D1%80%D0%BC). - -**Синтаксис** - -```sql -LinfDistance(tuple1, tuple2) -``` - -Синоним: `distanceLinf`. - -**Аргументы** - -- `tuple1` — первый кортеж. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — второй кортеж. [Tuple](../../sql-reference/data-types/tuple.md). - -**Возвращаемые значения** - -- Расстояние в норме Linf. - -Тип: [Float](../../sql-reference/data-types/float.md). - -**Пример** - -Запрос: - -```sql -SELECT LinfDistance((1, 2), (2, 3)); -``` - -Результат: - -```text -┌─LinfDistance((1, 2), (2, 3))─┐ -│ 1 │ -└──────────────────────────────┘ -``` - -## LpDistance {#lpdistance} - -Вычисляет расстояние между двумя точками (значения кортежей — координаты точек) в норме [Lp](https://ru.wikipedia.org/wiki/%D0%9D%D0%BE%D1%80%D0%BC%D0%B0_(%D0%BC%D0%B0%D1%82%D0%B5%D0%BC%D0%B0%D1%82%D0%B8%D0%BA%D0%B0)#%D0%9D%D0%B5%D0%BA%D0%BE%D1%82%D0%BE%D1%80%D1%8B%D0%B5_%D0%B2%D0%B8%D0%B4%D1%8B_%D0%BC%D0%B0%D1%82%D1%80%D0%B8%D1%87%D0%BD%D1%8B%D1%85_%D0%BD%D0%BE%D1%80%D0%BC). - -**Синтаксис** - -```sql -LpDistance(tuple1, tuple2, p) -``` - -Синоним: `distanceLp`. - -**Аргументы** - -- `tuple1` — первый кортеж. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — второй кортеж. [Tuple](../../sql-reference/data-types/tuple.md). -- `p` — степень. Возможные значение: любое число из промежутка [1;inf). [UInt](../../sql-reference/data-types/int-uint.md) или [Float](../../sql-reference/data-types/float.md). - -**Возвращаемое значение** - -- Расстояние в норме Lp. - -Тип: [Float](../../sql-reference/data-types/float.md). - -**Пример** - -Запрос: - -```sql -SELECT LpDistance((1, 2), (2, 3), 3); -``` - -Результат: - -```text -┌─LpDistance((1, 2), (2, 3), 3)─┐ -│ 1.2599210498948732 │ -└───────────────────────────────┘ -``` - -## L1Normalize {#l1normalize} - -Вычисляет единичный вектор для исходного вектора (значения кортежа — координаты вектора) в норме L1 ([расстояние городских кварталов](https://ru.wikipedia.org/wiki/%D0%A0%D0%B0%D1%81%D1%81%D1%82%D0%BE%D1%8F%D0%BD%D0%B8%D0%B5_%D0%B3%D0%BE%D1%80%D0%BE%D0%B4%D1%81%D0%BA%D0%B8%D1%85_%D0%BA%D0%B2%D0%B0%D1%80%D1%82%D0%B0%D0%BB%D0%BE%D0%B2)). - -**Синтаксис** - -```sql -L1Normalize(tuple) -``` - -Синоним: `normalizeL1`. - -**Аргументы** - -- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). - -**Возвращаемое значение** - -- Единичный вектор. - -Тип: кортеж [Tuple](../../sql-reference/data-types/tuple.md) значений [Float](../../sql-reference/data-types/float.md). - -**Пример** - -Запрос: - -```sql -SELECT L1Normalize((1, 2)); -``` - -Результат: - -```text -┌─L1Normalize((1, 2))─────────────────────┐ -│ (0.3333333333333333,0.6666666666666666) │ -└─────────────────────────────────────────┘ -``` - -## L2Normalize {#l2normalize} - -Вычисляет единичный вектор для исходного вектора (значения кортежа — координаты вектора) в норме L2 ([Евклидово пространство](https://ru.wikipedia.org/wiki/%D0%95%D0%B2%D0%BA%D0%BB%D0%B8%D0%B4%D0%BE%D0%B2%D0%BE_%D0%BF%D1%80%D0%BE%D1%81%D1%82%D1%80%D0%B0%D0%BD%D1%81%D1%82%D0%B2%D0%BE). - -**Синтаксис** - -```sql -L2Normalize(tuple) -``` - -Синоним: `normalizeL1`. - -**Аргументы** - -- `tuple` — кортеж. [Tuple](../../sql-reference/data-types/tuple.md). - -**Возвращаемое значение** - -- Единичный вектор. - -Тип: кортеж [Tuple](../../sql-reference/data-types/tuple.md) значений [Float](../../sql-reference/data-types/float.md). - -**Пример** - -Запрос: - -```sql -SELECT L2Normalize((3, 4)); -``` - -Результат: - -```text -┌─L2Normalize((3, 4))─┐ -│ (0.6,0.8) │ -└─────────────────────┘ -``` - -## LinfNormalize {#linfnormalize} - -Вычисляет единичный вектор для исходного вектора (значения кортежа — координаты вектора) в норме [Linf](https://ru.wikipedia.org/wiki/%D0%9D%D0%BE%D1%80%D0%BC%D0%B0_(%D0%BC%D0%B0%D1%82%D0%B5%D0%BC%D0%B0%D1%82%D0%B8%D0%BA%D0%B0)#%D0%9D%D0%B5%D0%BA%D0%BE%D1%82%D0%BE%D1%80%D1%8B%D0%B5_%D0%B2%D0%B8%D0%B4%D1%8B_%D0%BC%D0%B0%D1%82%D1%80%D0%B8%D1%87%D0%BD%D1%8B%D1%85_%D0%BD%D0%BE%D1%80%D0%BC). - -**Синтаксис** - -```sql -LinfNormalize(tuple) -``` - -Синоним: `normalizeLinf `. - -**Аргументы** - -- `tuple` — кортеж. [Tuple](../../sql-reference/data-types/tuple.md). - -**Возвращаемое значение** - -- Единичный вектор. - -Тип: кортеж [Tuple](../../sql-reference/data-types/tuple.md) значений [Float](../../sql-reference/data-types/float.md). - -**Пример** - -Запрос: - -```sql -SELECT LinfNormalize((3, 4)); -``` - -Результат: - -```text -┌─LinfNormalize((3, 4))─┐ -│ (0.75,1) │ -└───────────────────────┘ -``` - -## LpNormalize {#lpnormalize} - -Вычисляет единичный вектор для исходного вектора (значения кортежа — координаты вектора) в норме [Lp](https://ru.wikipedia.org/wiki/%D0%9D%D0%BE%D1%80%D0%BC%D0%B0_(%D0%BC%D0%B0%D1%82%D0%B5%D0%BC%D0%B0%D1%82%D0%B8%D0%BA%D0%B0)#%D0%9D%D0%B5%D0%BA%D0%BE%D1%82%D0%BE%D1%80%D1%8B%D0%B5_%D0%B2%D0%B8%D0%B4%D1%8B_%D0%BC%D0%B0%D1%82%D1%80%D0%B8%D1%87%D0%BD%D1%8B%D1%85_%D0%BD%D0%BE%D1%80%D0%BC). - -**Синтаксис** - -```sql -LpNormalize(tuple, p) -``` - -Синоним: `normalizeLp `. - -**Аргументы** - -- `tuple` — кортеж. [Tuple](../../sql-reference/data-types/tuple.md). -- `p` — степень. Возможные значение: любое число из промежутка [1;inf). [UInt](../../sql-reference/data-types/int-uint.md) или [Float](../../sql-reference/data-types/float.md). - -**Возвращаемое значение** - -- Единичный вектор. - -Тип: кортеж [Tuple](../../sql-reference/data-types/tuple.md) значений [Float](../../sql-reference/data-types/float.md). - -**Пример** - -Запрос: - -```sql -SELECT LpNormalize((3, 4),5); -``` - -Результат: - -```text -┌─LpNormalize((3, 4), 5)──────────────────┐ -│ (0.7187302630182624,0.9583070173576831) │ -└─────────────────────────────────────────┘ -``` - -## cosineDistance {#cosinedistance} - -Вычисляет косинусную разницу двух векторов (значения кортежей — координаты векторов). Чем меньше возвращаемое значение, тем больше сходство между векторами. - -**Синтаксис** - -```sql -cosineDistance(tuple1, tuple2) -``` - -**Аргументы** - -- `tuple1` — первый кортеж. [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple2` — второй кортеж. [Tuple](../../sql-reference/data-types/tuple.md). - -**Возвращаемые значения** - -- Результат разности между единицей и косинуса угла между векторами. - -Тип: [Float](../../sql-reference/data-types/float.md). - -**Пример** - -Запрос: - -```sql -SELECT cosineDistance((1, 2),(2,3)); -``` - -Результат: - -```text -┌─cosineDistance((1, 2), (2, 3))─┐ -│ 0.007722123286332261 │ -└────────────────────────────────┘ -``` diff --git a/docs/ru/sql-reference/operators/index.md b/docs/ru/sql-reference/operators/index.md index 08054c28ab8..98f6873f712 100644 --- a/docs/ru/sql-reference/operators/index.md +++ b/docs/ru/sql-reference/operators/index.md @@ -18,30 +18,20 @@ toc_title: "Операторы" `-a` - функция `negate(a)`. -Для отрицания чисел в кортеже: [tupleNegate](../../sql-reference/functions/tuple-functions.md#tuplenegate). - ## Операторы умножения и деления {#operatory-umnozheniia-i-deleniia} `a * b` - функция `multiply(a, b)` -Для умножения кортежа на число: [tupleMultiplyByNumber](../../sql-reference/functions/tuple-functions.md#tuplemultiplybynumber), для скалярного произведения: [dotProduct](../../sql-reference/functions/tuple-functions.md#dotproduct). - `a / b` - функция `divide(a, b)` -Для деления кортежа на число: [tupleDivideByNumber](../../sql-reference/functions/tuple-functions.md#tupledividebynumber). - `a % b` - функция `modulo(a, b)` ## Операторы сложения и вычитания {#operatory-slozheniia-i-vychitaniia} `a + b` - функция `plus(a, b)` -Для сложения кортежей: [tuplePlus](../../sql-reference/functions/tuple-functions.md#tupleplus). - `a - b` - функция `minus(a, b)` -Для вычитания кортежей: [tupleMinus](../../sql-reference/functions/tuple-functions.md#tupleminus). - ## Операторы сравнения {#operatory-sravneniia} `a = b` - функция `equals(a, b)` From 8e680b068ad4d57db3701a953943451959a8db76 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 25 Oct 2021 16:53:23 +0300 Subject: [PATCH 071/173] Add comment with fix explanation --- src/Common/ProgressIndication.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 5a3f8cfc350..db1daaac415 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -122,6 +122,9 @@ ProgressIndication::MemoryUsage ProgressIndication::getMemoryUsage() const [](MemoryUsage const & acc, auto const & host_data) { UInt64 host_usage = 0; + // In ProfileEvents packets thread id 0 specifies common profiling information + // for all threads executing current query on specific host. So instead of summing per thread + // memory consumption it's enough to look for data with thread id 0. if (auto it = host_data.second.find(ZERO); it != host_data.second.end()) host_usage = it->second.memory_usage; return MemoryUsage{.total = acc.total + host_usage, .max = std::max(acc.max, host_usage)}; From 94088f8197f0bd530e04362c5132bf2d84d20d8c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 25 Oct 2021 17:01:23 +0300 Subject: [PATCH 072/173] minor fixes --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 9 +++-- .../MergeTree/ReplicatedMergeTreeQueue.h | 3 +- src/Storages/StorageReplicatedMergeTree.cpp | 37 +++++++++++++++---- 3 files changed, 36 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 6c1cd04d4ae..985d54e8aee 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1004,7 +1004,7 @@ bool ReplicatedMergeTreeQueue::checkReplaceRangeCanBeRemoved(const MergeTreePart void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( zkutil::ZooKeeperPtr zookeeper, const MergeTreePartInfo & part_info, - const std::optional & current) + const std::optional & covering_entry) { /// TODO is it possible to simplify it? Queue to_wait; @@ -1015,8 +1015,9 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( /// Remove operations with parts, contained in the range to be deleted, from the queue. std::unique_lock lock(state_mutex); - [[maybe_unused]] bool called_from_alter_query_directly = current && current->replace_range_entry && current->replace_range_entry->columns_version < 0; - [[maybe_unused]] bool called_for_broken_part = !current; + [[maybe_unused]] bool called_from_alter_query_directly = covering_entry && covering_entry->replace_range_entry + && covering_entry->replace_range_entry->columns_version < 0; + [[maybe_unused]] bool called_for_broken_part = !covering_entry; assert(currently_executing_drop_or_replace_range || called_from_alter_query_directly || called_for_broken_part); for (Queue::iterator it = queue.begin(); it != queue.end();) @@ -1028,7 +1029,7 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( type == LogEntry::MUTATE_PART; bool simple_op_covered = is_simple_producing_op && part_info.contains(MergeTreePartInfo::fromPartName((*it)->new_part_name, format_version)); - bool replace_range_covered = current && checkReplaceRangeCanBeRemoved(part_info, *it, *current); + bool replace_range_covered = covering_entry && checkReplaceRangeCanBeRemoved(part_info, *it, *covering_entry); if (simple_op_covered || replace_range_covered) { if ((*it)->currently_executing) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index c059ff7a072..273ac21eebc 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -325,9 +325,10 @@ public: /** Remove the action from the queue with the parts covered by part_name (from ZK and from the RAM). * And also wait for the completion of their execution, if they are now being executed. + * covering_entry is as an entry that caused removal of entries in range (usually, DROP_RANGE) */ void removePartProducingOpsInRange(zkutil::ZooKeeperPtr zookeeper, const MergeTreePartInfo & part_info, - const std::optional & current); + const std::optional & covering_entry); /** In the case where there are not enough parts to perform the merge in part_name * - move actions with merged parts to the end of the queue diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 052303e3627..f7bccb7e823 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2319,6 +2319,13 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo LogEntryPtr parsed_entry = {}; }; + /// We got log pointer and list of queue entries of source replica. + /// At first we will get queue entries and then we will get list of active parts of source replica + /// to enqueue fetches for missing parts. If source replica executes and removes some entry concurrently + /// we will see produced part (or covering part) in replicas/source/parts and will enqueue fetch. + /// We will try to parse queue entries before copying them + /// to avoid creation of excessive and duplicating entries in our queue. + /// See also removePartAndEnqueueFetch(...) std::vector source_queue; ActiveDataPartSet get_part_set{format_version}; ActiveDataPartSet drop_range_set{format_version}; @@ -2358,11 +2365,15 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo info.parsed_entry->znode_name = source_queue_names[i]; - if (info.parsed_entry->type == LogEntry::GET_PART) - get_part_set.add(info.parsed_entry->new_part_name); - if (info.parsed_entry->type == LogEntry::DROP_RANGE) drop_range_set.add(info.parsed_entry->new_part_name); + + if (info.parsed_entry->type == LogEntry::GET_PART) + { + String maybe_covering_drop_range = drop_range_set.getContainingPart(info.parsed_entry->new_part_name); + if (maybe_covering_drop_range.empty()) + get_part_set.add(info.parsed_entry->new_part_name); + } } } @@ -2396,7 +2407,8 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo Coordination::Stat is_lost_stat_new; zookeeper->get(fs::path(source_path) / "is_lost", &is_lost_stat_new); if (is_lost_stat_new.version != source_is_lost_stat.version) - throw Exception(ErrorCodes::REPLICA_STATUS_CHANGED, "Cannot clone {}, because it suddenly become lost", source_replica); + throw Exception(ErrorCodes::REPLICA_STATUS_CHANGED, "Cannot clone {}, because it suddenly become lost " + "or removed broken part from ZooKeeper", source_replica); } tryRemovePartsFromZooKeeperWithRetries(parts_to_remove_from_zk); @@ -2431,7 +2443,8 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo /// Avoid creation of GET_PART entries which covered by another GET_PART or DROP_RANGE /// and creation of multiple entries with the same new_part_name. - auto should_ignore_log_entry = [&] (const String & part_name, const String & log_msg_context) -> bool + auto should_ignore_log_entry = [&drop_range_set, &get_part_set, this] (std::unordered_set & created_gets, + const String & part_name, const String & log_msg_context) -> bool { /// We should not create entries covered by DROP_RANGE, because we will remove them anyway (kind of optimization). String covering_drop_range = drop_range_set.getContainingPart(part_name); @@ -2460,7 +2473,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo /// but before we copied its active parts set. In this case we will GET_PART entry in our queue /// and later will pull the original GET_PART from replication log. /// It should not cause any issues, but it does not allow to get rid of duplicated entries and add an assertion. - if (created_get_parts.count(part_name)) + if (created_gets.count(part_name)) { /// NOTE It would be better to copy log entry instead of creating GET_PART /// if there are GET_PART and log entry of other type with the same new_part_name. @@ -2474,7 +2487,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo for (const String & name : active_parts) { - if (should_ignore_log_entry(name, "Not fetching")) + if (should_ignore_log_entry(created_get_parts, name, "Not fetching")) continue; LogEntry log_entry; @@ -2532,7 +2545,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo const String & entry_name = entry_info.parsed_entry->znode_name; const auto & entry_type = entry_info.parsed_entry->type; - if (should_ignore_log_entry(part_name, fmt::format("Not copying {} {} ", entry_name, entry_type))) + if (should_ignore_log_entry(created_get_parts, part_name, fmt::format("Not copying {} {} ", entry_name, entry_type))) continue; if (entry_info.parsed_entry->type == LogEntry::GET_PART) @@ -3317,6 +3330,14 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n Coordination::Stat stat; if (zookeeper->exists(part_path, &stat)) { + /// Update version of /is_lost node to avoid race condition with cloneReplica(...). + /// cloneReplica(...) expects that if some entry was executed, then its new_part_name is added to /parts, + /// but we are going to remove it from /parts and add to queue again. + Coordination::Stat is_lost_stat; + String is_lost_value = zookeeper->get(replica_path + "/is_lost", &is_lost_stat); + assert(is_lost_value == "0"); + ops.emplace_back(zkutil::makeSetRequest(replica_path + "/is_lost", is_lost_value, is_lost_stat.version)); + part_create_time = stat.ctime / 1000; removePartFromZooKeeper(part_name, ops, stat.numChildren > 0); } From 55e887728e8736e14c4d35b6cced0c38c780f62f Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Mon, 25 Oct 2021 12:35:09 +0300 Subject: [PATCH 073/173] Retry on Yandex.S3 throttle --- contrib/aws | 2 +- .../configs/config.d/storage_conf.xml | 16 ++++++++++ .../s3_endpoint/endpoint.py | 19 ++++++++++++ .../test_merge_tree_s3_failover/test.py | 30 +++++++++++++++++++ 4 files changed, 66 insertions(+), 1 deletion(-) diff --git a/contrib/aws b/contrib/aws index 06aa8759d17..00b03604543 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit 06aa8759d17f2032ffd5efa83969270ca9ac727b +Subproject commit 00b03604543367d7e310cb0993973fdcb723ea79 diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml index f5e350e5b6b..32d78468a71 100644 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml @@ -12,6 +12,15 @@ 0 + + s3 + + http://resolver:8080/root/data/ + minio + minio123 + + true + @@ -32,6 +41,13 @@ + + +
+ s3_retryable +
+
+
diff --git a/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py b/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py index 1754b1f175c..3f219b6ba57 100644 --- a/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py +++ b/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py @@ -18,6 +18,16 @@ def fail_request(_request_number): return 'OK' +@route('/throttle_request/<_request_number>') +def fail_request(_request_number): + request_number = int(_request_number) + if request_number > 0: + cache['throttle_request_number'] = request_number + else: + cache.pop('throttle_request_number', None) + return 'OK' + + # Handle for MultipleObjectsDelete. @route('/<_bucket>', ['POST']) def delete(_bucket): @@ -37,6 +47,15 @@ def server(_bucket, _path): response.content_type = 'text/xml' return 'ExpectedErrorExpected Errortxfbd566d03042474888193-00608d7537' + if cache.get('throttle_request_number', None): + request_number = cache.pop('throttle_request_number') - 1 + if request_number > 0: + cache['throttle_request_number'] = request_number + else: + response.status = 429 + response.content_type = 'text/xml' + return 'TooManyRequestsExceptionPlease reduce your request rate.txfbd566d03042474888193-00608d7538' + response.set_header("Location", "http://minio1:9001/" + _bucket + '/' + _path) response.status = 307 return 'Redirected' diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index 56d9441aba6..b6b47417523 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -38,6 +38,12 @@ def fail_request(cluster, request): assert response == 'OK', 'Expected "OK", but got "{}"'.format(response) +def throttle_request(cluster, request): + response = cluster.exec_in_container(cluster.get_container_id('resolver'), + ["curl", "-s", "http://resolver:8080/throttle_request/{}".format(request)]) + assert response == 'OK', 'Expected "OK", but got "{}"'.format(response) + + @pytest.fixture(scope="module") def cluster(): try: @@ -186,3 +192,27 @@ def test_move_failover(cluster): # Ensure data is not corrupted. assert node.query("CHECK TABLE s3_failover_test") == '1\n' assert node.query("SELECT id,data FROM s3_failover_test FORMAT Values") == "(0,'data'),(1,'data')" + + +# Check that throttled request retries and does not cause an error on disk with default `retry_attempts` (>0) +def test_throttle_retry(cluster): + node = cluster.instances["node"] + + node.query( + """ + CREATE TABLE s3_throttle_retry_test ( + id Int64 + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS storage_policy='s3_retryable' + """ + ) + + data = "(42)" + node.query("INSERT INTO s3_throttle_retry_test VALUES {}".format(data)) + + throttle_request(cluster, 1) + + assert node.query(""" + SELECT * FROM s3_throttle_retry_test + """) == '42\n' From 5cdd9a4b99c84f44e105c064a955356c02a9749a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 25 Oct 2021 19:00:55 +0300 Subject: [PATCH 074/173] fix intersecting parts in parts_to_do --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index f397e73aa31..29a13970b3e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -203,9 +203,15 @@ void ReplicatedMergeTreeQueue::insertUnlocked( drop_ranges.addDropRange(entry); /// DROP PART remove parts, so we remove it from virtual parts to - /// preserve invariant virtual_parts = current_parts + queue + /// preserve invariant virtual_parts = current_parts + queue. + /// Also remove it from parts_to_do to avoid intersecting parts in parts_to_do + /// if fast replica will execute DROP PART and assign a merge that contains dropped blocks. if (entry->isDropPart(format_version)) - virtual_parts.removePartAndCoveredParts(*entry->getDropRange(format_version)); + { + String drop_part_name = *entry->getDropRange(format_version); + virtual_parts.removePartAndCoveredParts(drop_part_name); + removeCoveredPartsFromMutations(drop_part_name, /*remove_part = */ true, /*remove_covered_parts = */ true); + } queue.push_front(entry); } From 5231e8b8d9dfdea1f95e4e78a87c46b1d870e256 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 25 Oct 2021 20:12:29 +0300 Subject: [PATCH 075/173] HashedDictionaries added read performance tests --- tests/performance/hashed_array_dictionary.xml | 10 ++++++++++ tests/performance/hashed_dictionary.xml | 10 ++++++++++ 2 files changed, 20 insertions(+) diff --git a/tests/performance/hashed_array_dictionary.xml b/tests/performance/hashed_array_dictionary.xml index c992be117d3..80a565e67b0 100644 --- a/tests/performance/hashed_array_dictionary.xml +++ b/tests/performance/hashed_array_dictionary.xml @@ -103,6 +103,11 @@ FORMAT Null; + + SELECT * FROM simple_key_hashed_array_dictionary + FORMAT Null; + + WITH (rand64() % toUInt64({elements_count}), toString(rand64() % toUInt64({elements_count}))) as key SELECT dictGet('default.complex_key_hashed_array_dictionary', {column_name}, key) @@ -119,6 +124,11 @@ FORMAT Null; + + SELECT * FROM complex_key_hashed_array_dictionary + FORMAT Null; + + DROP TABLE IF EXISTS simple_key_hashed_array_dictionary_source_table; DROP TABLE IF EXISTS complex_key_hashed_array_dictionary_source_table; diff --git a/tests/performance/hashed_dictionary.xml b/tests/performance/hashed_dictionary.xml index fcc8c1bd539..cf1cdac6df1 100644 --- a/tests/performance/hashed_dictionary.xml +++ b/tests/performance/hashed_dictionary.xml @@ -103,6 +103,11 @@ FORMAT Null; + + SELECT * FROM simple_key_hashed_dictionary + FORMAT Null; + + WITH (rand64() % toUInt64({elements_count}), toString(rand64() % toUInt64({elements_count}))) as key SELECT dictGet('default.complex_key_hashed_dictionary', {column_name}, key) @@ -119,6 +124,11 @@ FORMAT Null; + + SELECT * FROM complex_key_hashed_dictionary + FORMAT Null; + + DROP TABLE IF EXISTS simple_key_hashed_dictionary_source_table; DROP TABLE IF EXISTS complex_key_hashed_dictionary_source_table; From 7fe028245ea9e395dc16b7c1adf07b2ea52fa479 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Oct 2021 21:12:45 +0300 Subject: [PATCH 076/173] Do not save preprocessed config for clickhouse-local It seems that there is no purpose on doing this, and it just produce an error due to permission errors (i.e. before this patch if you will pass small config with some overrides then it will try to write to /var/lib/clickhouse by default). --- programs/local/LocalServer.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 69f7820909a..b45bb991887 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -179,7 +179,6 @@ void LocalServer::initialize(Poco::Util::Application & self) ConfigProcessor config_processor(config_path, false, true); config_processor.setConfigPath(fs::path(config_path).parent_path()); auto loaded_config = config_processor.loadConfig(); - config_processor.savePreprocessedConfig(loaded_config, loaded_config.configuration->getString("path", ".")); config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false); } @@ -380,7 +379,6 @@ void LocalServer::setupUsers() const auto users_config_path = config().getString("users_config", config().getString("config-file", "config.xml")); ConfigProcessor config_processor(users_config_path); const auto loaded_config = config_processor.loadConfig(); - config_processor.savePreprocessedConfig(loaded_config, config().getString("path", DBMS_DEFAULT_PATH)); users_config = loaded_config.configuration; } else From 096eaa1008c8104da37a147ece75831b7ca7309a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Oct 2021 21:12:45 +0300 Subject: [PATCH 077/173] Initialize custom TLDs in clickhouse-local --- programs/local/LocalServer.cpp | 9 +++++++- ...110_clickhouse_local_custom_tld.config.xml | 22 +++++++++++++++++++ ...2110_clickhouse_local_custom_tld.reference | 1 + .../02110_clickhouse_local_custom_tld.sh | 21 ++++++++++++++++++ .../02110_clickhouse_local_custom_tld.tld.dat | 1 + 5 files changed, 53 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02110_clickhouse_local_custom_tld.config.xml create mode 100644 tests/queries/0_stateless/02110_clickhouse_local_custom_tld.reference create mode 100755 tests/queries/0_stateless/02110_clickhouse_local_custom_tld.sh create mode 100644 tests/queries/0_stateless/02110_clickhouse_local_custom_tld.tld.dat diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index b45bb991887..2a2fe4a78c8 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -18,7 +18,9 @@ #include #include #include +#include #include +#include #include #include #include @@ -35,7 +37,6 @@ #include #include #include -#include #include namespace fs = std::filesystem; @@ -283,6 +284,11 @@ void LocalServer::tryInitPath() global_context->setFlagsPath(path + "flags"); global_context->setUserFilesPath(""); // user's files are everywhere + + /// top_level_domains_lists + const std::string & top_level_domains_path = config().getString("top_level_domains_path", path + "top_level_domains/"); + if (!top_level_domains_path.empty()) + TLDListsHolder::getInstance().parseConfig(fs::path(top_level_domains_path) / "", config()); } @@ -671,6 +677,7 @@ void LocalServer::addOptions(OptionsDescription & options_description) ("no-system-tables", "do not attach system tables (better startup time)") ("path", po::value(), "Storage path") + ("top_level_domains_path", po::value(), "Path to lists with custom TLDs") ; } diff --git a/tests/queries/0_stateless/02110_clickhouse_local_custom_tld.config.xml b/tests/queries/0_stateless/02110_clickhouse_local_custom_tld.config.xml new file mode 100644 index 00000000000..0baa5b32ef8 --- /dev/null +++ b/tests/queries/0_stateless/02110_clickhouse_local_custom_tld.config.xml @@ -0,0 +1,22 @@ + + + + + + + + + ::/0 + + default + default + + + + + + + + 02110_clickhouse_local_custom_tld.tld.dat + + diff --git a/tests/queries/0_stateless/02110_clickhouse_local_custom_tld.reference b/tests/queries/0_stateless/02110_clickhouse_local_custom_tld.reference new file mode 100644 index 00000000000..bf62392aeee --- /dev/null +++ b/tests/queries/0_stateless/02110_clickhouse_local_custom_tld.reference @@ -0,0 +1 @@ +biz.ss kernel.biz.ss diff --git a/tests/queries/0_stateless/02110_clickhouse_local_custom_tld.sh b/tests/queries/0_stateless/02110_clickhouse_local_custom_tld.sh new file mode 100755 index 00000000000..02a454957d4 --- /dev/null +++ b/tests/queries/0_stateless/02110_clickhouse_local_custom_tld.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function clickhouse_local() +{ + local opts=( + --config "$CURDIR/$(basename "${BASH_SOURCE[0]}" .sh).config.xml" + --top_level_domains_path "$CURDIR" + ) + $CLICKHOUSE_LOCAL "${opts[@]}" "$@" +} + +# -- biz.ss is not in the default TLD list, hence: +clickhouse_local -q " + select + cutToFirstSignificantSubdomain('foo.kernel.biz.ss'), + cutToFirstSignificantSubdomainCustom('foo.kernel.biz.ss', 'public_suffix_list') +" |& grep -v -e 'Processing configuration file' diff --git a/tests/queries/0_stateless/02110_clickhouse_local_custom_tld.tld.dat b/tests/queries/0_stateless/02110_clickhouse_local_custom_tld.tld.dat new file mode 100644 index 00000000000..510e6dd9ec0 --- /dev/null +++ b/tests/queries/0_stateless/02110_clickhouse_local_custom_tld.tld.dat @@ -0,0 +1 @@ +biz.ss From f41cf281e4a29e9d703bde1e29c74a7433f1a208 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Oct 2021 21:14:16 +0300 Subject: [PATCH 078/173] Add QueryProfilerRuns profile event This will help to investigate query latency spikes. --- src/Common/ProfileEvents.cpp | 3 ++- src/Common/QueryProfiler.cpp | 2 ++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 941a3ab0896..44adbccc089 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -227,7 +227,8 @@ M(CreatedHTTPConnections, "Total amount of created HTTP connections (counter increase every time connection is created).") \ \ M(CannotWriteToWriteBufferDiscard, "Number of stack traces dropped by query profiler or signal handler because pipe is full or cannot write to pipe.") \ - M(QueryProfilerSignalOverruns, "Number of times we drop processing of a signal due to overrun plus the number of signals that OS has not delivered due to overrun.") \ + M(QueryProfilerSignalOverruns, "Number of times we drop processing of a query profiler signal due to overrun plus the number of signals that OS has not delivered due to overrun.") \ + M(QueryProfilerRuns, "Number of times QueryProfiler had been run.") \ \ M(CreatedLogEntryForMerge, "Successfully created log entry to merge parts in ReplicatedMergeTree.") \ M(NotCreatedLogEntryForMerge, "Log entry to merge parts in ReplicatedMergeTree is not created due to concurrent log update by another replica.") \ diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index aa40226093a..77b8d0dda5e 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -15,6 +15,7 @@ namespace ProfileEvents { extern const Event QueryProfilerSignalOverruns; + extern const Event QueryProfilerRuns; } namespace DB @@ -60,6 +61,7 @@ namespace const StackTrace stack_trace(signal_context); TraceCollector::collect(trace_type, stack_trace, 0); + ProfileEvents::increment(ProfileEvents::QueryProfilerRuns); errno = saved_errno; } From 0b7a8c98871a704e9d093be006bef4d6dfce961a Mon Sep 17 00:00:00 2001 From: gyuton Date: Mon, 25 Oct 2021 22:35:14 +0300 Subject: [PATCH 079/173] minor fix --- docs/en/operations/settings/settings.md | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 3eb39907991..47e0a3006da 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3834,7 +3834,7 @@ Default value: `0`. ## async_insert_threads {#async-insert-threads} -The maximum number of threads for backgroung data parsing and insertion. +The maximum number of threads for background data parsing and insertion. Possible values: @@ -3858,6 +3858,8 @@ Default value: `1`. The timeout in seconds for waiting for processing of asynchronous insertion. +Possible values: + - Positive integer. - 0 — Disabled. @@ -3878,6 +3880,8 @@ Default value: `1000000`. The maximum timeout in milliseconds since the first `INSERT` query before inserting collected data. +Possible values: + - Positive integer. - 0 — Timeout disabled. @@ -3887,6 +3891,8 @@ Default value: `200`. The maximum timeout in milliseconds since the last `INSERT` query before dumping collected data. If enabled, the settings prolongs the [async_insert_busy_timeout](#async-insert-busy-timeout) with every `INSERT` query as long as [async_insert_max_data_size](#async-insert-max-data-size) is not exceeded. +Possible values: + - Positive integer. - 0 — Timeout disabled. From c08f6cae0523e32b49c6a3cb49bb79f8668b4c26 Mon Sep 17 00:00:00 2001 From: gyuton Date: Mon, 25 Oct 2021 22:39:47 +0300 Subject: [PATCH 080/173] translated to Russian --- .../sql-reference/statements/insert-into.md | 10 +++ docs/ru/operations/settings/settings.md | 84 +++++++++++++++++++ .../sql-reference/statements/insert-into.md | 11 +++ 3 files changed, 105 insertions(+) diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index c44f7435089..ad5ab705937 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -120,3 +120,13 @@ Performance will not decrease if: - You upload data that is usually sorted by time. It's also possible to asynchronously insert data in small but frequent inserts from multiple concurrent connections. The data from such insertions is combined into batches and then safely inserted into a table. To enable the asynchronous mode, switch on the [async_insert](../../operations/settings/settings.md#async-insert) setting. Note that asynchronous insertions are supported only over HTTP protocol, and deduplication is not supported for them. + +**See Also** + +- [async_insert](../../operations/settings/settings.md#async-insert) +- [async_insert_threads](../../operations/settings/settings.md#async-insert-threads) +- [wait_for_async_insert](../../operations/settings/settings.md#wait-for-async-insert) +- [wait_for_async_insert_timeout](../../operations/settings/settings.md#wait-for-async-insert-timeout) +- [async_insert_max_data_size](../../operations/settings/settings.md#async-insert-max-data-size) +- [async_insert_busy_timeout](../../operations/settings/settings.md#async-insert-busy-timeout) +- [async_insert_stale_timeout](../../operations/settings/settings.md#async-insert-stale-timeout) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 71b6ac513fc..fb84913c0db 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3640,3 +3640,87 @@ SELECT * FROM positional_arguments ORDER BY 2,3; **См. также** - настройка [optimize_move_to_prewhere](#optimize_move_to_prewhere) + +## async_insert {#async-insert} + +Включает или отключает асинхронные вставки для нескольких параллельных соединений. Работает только для вставок по протоколу HTTP. Обратите внимание, что при таких вставках дедупликация не производится. + +Если включено, данные собираются в пачки перед вставкой в таблицу. Это позволяет производить мелкие и частые вставки в Clickhouse (до 15000 запросов в секунду) без промежуточных таблиц. + +Вставка данных происходит либо как только привышен [async_insert_max_data_size](#async-insert-max-data-size), либо через [async_insert_busy_timeout](#async-insert-busy-timeout) миллисекунд после первого запроса `INSERT`. Если [async_insert_stale_timeout](#async-insert-stale-timeout) задано ненулевое значение, то данные вставляются через `async_insert_stale_timeout` после последнего запроса. + +Если включен параметр [wait_for_async_insert](#wait-for-async-insert), каждый клиент ждет, пока данные будут сброшены в таблицу. Иначе запрос будет обработан почти моментально, даже если данные еще не вставлены. + +Возможные значения: + +- 0 — вставки производятся синхронно, один запрос за другим. +- 1 — включены множественные асинхронные вставки. + +Значение по умолчанию: `0`. + +## async_insert_threads {#async-insert-threads} + +Максимальное число потоков для фоновой обработки и вставки данных. + +Возможные значения: + +- положительное целое число. +- 0 — асинхронные вставки отключены. + +Значение по умолчанию: `16`. + +## wait_for_async_insert {#wait-for-async-insert} + +Включает или отключает ожидание обработки асинхронных вставок. Если включено, клиент выведет `OK` только после того, как данные вставлены. Иначе будет выведен `OK`, даже если вставка не произошла. + +Возможные значения: + +- 0 — отключено. +- 1 — включено. + +Значение по умолчанию: `1`. + +## wait_for_async_insert_timeout {#wait-for-async-insert-timeout} + +Время ожидания в секундах, выделяемое для обработки асинхронной вставки. + +Возможные значения: + +- Положительное целое число. +- 0 — ожидание отключено. + +Значение по умолчанию: [lock_acquire_timeout](#lock_acquire_timeout). + +## async_insert_max_data_size {#async-insert-max-data-size} + +Максимальный размер необработанных данных (в байтах), собранных за запрос, перед их вставкой. + +Возможные значения: + +- Положительное целое число. +- 0 — асинхронные вставки отключены. + +Значение по умолчанию: `1000000`. + +## async_insert_busy_timeout {#async-insert-busy-timeout} + +Максимальное время ожидания в миллисекундах после первого запроса `INSERT` и перед вставкой данных. + +Возможные значения: + +- Положительное целое число. +- 0 — ожидание отключено. + +Значение по умолчанию: `200`. + +## async_insert_stale_timeout {#async-insert-stale-timeout} + +Максимальное время ожидания в миллисекундах после последнего запроса `INSERT` и перед вставкой данных. Если установлено ненулевое значение, [async_insert_busy_timeout](#async-insert-busy-timeout) будет продлеваться с каждым запросом `INSERT`, пока не будет превышен [async_insert_max_data_size](#async-insert-max-data-size). + +Возможные значения: + +- Положительное целое число. +- 0 — ожидание отключено. + +Значение по умолчанию: `0`. + diff --git a/docs/ru/sql-reference/statements/insert-into.md b/docs/ru/sql-reference/statements/insert-into.md index da9f3a11101..d7732cc58e1 100644 --- a/docs/ru/sql-reference/statements/insert-into.md +++ b/docs/ru/sql-reference/statements/insert-into.md @@ -121,3 +121,14 @@ INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... - Данные поступают в режиме реального времени. - Вы загружаете данные, которые как правило отсортированы по времени. +Также возможно вставлять данные асинхронно во множественных маленьких вставках от нескольких параллельных соединений. Данные от таких вставок сначала собираются в пачки, а потом вставляются в таблицу. Чтобы включить асинхронный режим, используйте настройку [async_insert](../../operations/settings/settings.md#async-insert). Обратите внимание, что асинхронные вставки поддерживаются только через протокол HTTP, а дедупликация при этом не производится. + +**См. также** + +- [async_insert](../../operations/settings/settings.md#async-insert) +- [async_insert_threads](../../operations/settings/settings.md#async-insert-threads) +- [wait_for_async_insert](../../operations/settings/settings.md#wait-for-async-insert) +- [wait_for_async_insert_timeout](../../operations/settings/settings.md#wait-for-async-insert-timeout) +- [async_insert_max_data_size](../../operations/settings/settings.md#async-insert-max-data-size) +- [async_insert_busy_timeout](../../operations/settings/settings.md#async-insert-busy-timeout) +- [async_insert_stale_timeout](../../operations/settings/settings.md#async-insert-stale-timeout) From 3d115536f53ac8dc9af4b508ccc6cbc4d2ea569e Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Tue, 26 Oct 2021 01:39:09 +0530 Subject: [PATCH 081/173] Improve filter bitmask generator function all in one with sse/avx2/avx512 --- src/CMakeLists.txt | 1 + src/Columns/ColumnDecimal.cpp | 38 ++++++++++----- src/Columns/ColumnFixedString.cpp | 77 ++----------------------------- src/Columns/ColumnVector.cpp | 72 ++--------------------------- src/Columns/ColumnsCommon.cpp | 61 +++++++++++++++--------- src/Columns/ColumnsCommon.h | 2 + 6 files changed, 76 insertions(+), 175 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 8e8c3703672..9e1f25c21d9 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -294,6 +294,7 @@ set_source_files_properties( Columns/ColumnFixedString.cpp Columns/ColumnsCommon.cpp Columns/ColumnVector.cpp + Columns/ColumnDecimal.cpp PROPERTIES COMPILE_FLAGS "${X86_INTRINSICS_FLAGS}") if(RE2_LIBRARY) diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index dc236fafbd9..48f890edaaf 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -237,25 +237,39 @@ ColumnPtr ColumnDecimal::filter(const IColumn::Filter & filt, ssize_t result_ const UInt8 * filt_end = filt_pos + size; const T * data_pos = data.data(); -#ifdef __SSE2__ - static constexpr size_t SIMD_BYTES = 16; - const __m128i zero16 = _mm_setzero_si128(); - const UInt8 * filt_end_sse = filt_pos + size / SIMD_BYTES * SIMD_BYTES; + /** A slightly more optimized version. + * Based on the assumption that often pieces of consecutive values + * completely pass or do not pass the filter. + * Therefore, we will optimistically check the parts of `SIMD_BYTES` values. + */ + static constexpr size_t SIMD_BYTES = 64; + const UInt8 * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; - while (filt_pos < filt_end_sse) + while (filt_pos < filt_end_aligned) { - UInt16 mask = _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast(filt_pos)), zero16)); - mask = ~mask; - while (mask) + UInt64 mask = Bytes64MaskToBits64Mask(filt_pos); + + if (0xffffffffffffffff == mask) { - size_t index = __builtin_ctz(mask); - res_data.push_back(*(data_pos + index)); - mask = mask & (mask - 1); + res_data.insert(data_pos, data_pos + SIMD_BYTES); } + else + { + while (mask) + { + size_t index = __builtin_ctzll(mask); + res_data.push_back(data_pos[index]); + #ifdef __BMI__ + mask = _blsr_u64(mask); + #else + mask = mask & (mask-1); + #endif + } + } + filt_pos += SIMD_BYTES; data_pos += SIMD_BYTES; } -#endif while (filt_pos < filt_end) { diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 729d82da41d..cc4e7b12a04 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -236,17 +236,15 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result * completely pass or do not pass the filter. * Therefore, we will optimistically check the parts of `SIMD_BYTES` values. */ -#if defined(__AVX512F__) && defined(__AVX512BW__) static constexpr size_t SIMD_BYTES = 64; - const __m512i zero64 = _mm512_setzero_epi32(); - const UInt8 * filt_end_avx512 = filt_pos + col_size / SIMD_BYTES * SIMD_BYTES; + const UInt8 * filt_end_aligned = filt_pos + col_size / SIMD_BYTES * SIMD_BYTES; const size_t chars_per_simd_elements = SIMD_BYTES * n; - while (filt_pos < filt_end_avx512) + while (filt_pos < filt_end_aligned) { - uint64_t mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero64, _MM_CMPINT_GT); + uint64_t mask = Bytes64MaskToBits64Mask(filt_pos); - if (0xFFFFFFFFFFFFFFFF == mask) + if (0xffffffffffffffff == mask) { res->chars.insert(data_pos, data_pos + chars_per_simd_elements); } @@ -269,73 +267,6 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result data_pos += chars_per_simd_elements; filt_pos += SIMD_BYTES; } -#elif defined(__AVX__) && defined(__AVX2__) - static constexpr size_t SIMD_BYTES = 32; - const __m256i zero32 = _mm256_setzero_si256(); - const UInt8 * filt_end_avx2 = filt_pos + col_size / SIMD_BYTES * SIMD_BYTES; - const size_t chars_per_simd_elements = SIMD_BYTES * n; - - while (filt_pos < filt_end_avx2) - { - uint32_t mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero32)); - - if (0xFFFFFFFF == mask) - { - res->chars.insert(data_pos, data_pos + chars_per_simd_elements); - } - else - { - size_t res_chars_size = res->chars.size(); - while (mask) - { - size_t index = __builtin_ctz(mask); - res->chars.resize(res_chars_size + n); - memcpySmallAllowReadWriteOverflow15(&res->chars[res_chars_size], data_pos + index * n, n); - res_chars_size += n; - #ifdef __BMI__ - mask = _blsr_u32(mask); - #else - mask = mask & (mask-1); - #endif - } - } - data_pos += chars_per_simd_elements; - filt_pos += SIMD_BYTES; - } - -#elif defined(__SSE2__) - - static constexpr size_t SIMD_BYTES = 16; - const __m128i zero16 = _mm_setzero_si128(); - const UInt8 * filt_end_sse = filt_pos + col_size / SIMD_BYTES * SIMD_BYTES; - const size_t chars_per_simd_elements = SIMD_BYTES * n; - - while (filt_pos < filt_end_sse) - { - UInt16 mask = _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast(filt_pos)), zero16)); - mask = ~mask; - - if (0xFFFF == mask) - { - res->chars.insert(data_pos, data_pos + chars_per_simd_elements); - } - else - { - size_t res_chars_size = res->chars.size(); - while (mask) - { - size_t index = __builtin_ctz(mask); - res->chars.resize(res_chars_size + n); - memcpySmallAllowReadWriteOverflow15(&res->chars[res_chars_size], data_pos + index * n, n); - res_chars_size += n; - mask = mask & (mask - 1); - } - } - data_pos += chars_per_simd_elements; - filt_pos += SIMD_BYTES; - } - -#endif size_t res_chars_size = res->chars.size(); while (filt_pos < filt_end) diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 4793b6bd9d5..b9079857aa4 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -310,21 +310,20 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s const UInt8 * filt_pos = filt.data(); const UInt8 * filt_end = filt_pos + size; const T * data_pos = data.data(); + /** A slightly more optimized version. * Based on the assumption that often pieces of consecutive values * completely pass or do not pass the filter. * Therefore, we will optimistically check the parts of `SIMD_BYTES` values. */ -#if defined(__AVX512F__) && defined(__AVX512BW__) static constexpr size_t SIMD_BYTES = 64; - const __m512i zero64 = _mm512_setzero_epi32(); - const UInt8 * filt_end_avx512 = filt_pos + size / SIMD_BYTES * SIMD_BYTES; + const UInt8 * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; - while (filt_pos < filt_end_avx512) + while (filt_pos < filt_end_aligned) { - UInt64 mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero64, _MM_CMPINT_GT); + UInt64 mask = Bytes64MaskToBits64Mask(filt_pos); - if (0xFFFFFFFFFFFFFFFF == mask) + if (0xffffffffffffffff == mask) { res_data.insert(data_pos, data_pos + SIMD_BYTES); } @@ -346,67 +345,6 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s data_pos += SIMD_BYTES; } -#elif defined(__AVX__) && defined(__AVX2__) - static constexpr size_t SIMD_BYTES = 32; - const __m256i zero32 = _mm256_setzero_si256(); - const UInt8 * filt_end_avx2 = filt_pos + size / SIMD_BYTES * SIMD_BYTES; - - while (filt_pos < filt_end_avx2) - { - UInt32 mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero32)); - - if (0xFFFFFFFF == mask) - { - res_data.insert(data_pos, data_pos + SIMD_BYTES); - } - else - { - while (mask) - { - size_t index = __builtin_ctz(mask); - res_data.push_back(data_pos[index]); - #ifdef __BMI__ - mask = _blsr_u32(mask); - #else - mask = mask & (mask-1); - #endif - } - } - - filt_pos += SIMD_BYTES; - data_pos += SIMD_BYTES; - } - -#elif defined(__SSE2__) - static constexpr size_t SIMD_BYTES = 16; - const __m128i zero16 = _mm_setzero_si128(); - const UInt8 * filt_end_sse = filt_pos + size / SIMD_BYTES * SIMD_BYTES; - - while (filt_pos < filt_end_sse) - { - UInt16 mask = _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast(filt_pos)), zero16)); - mask = ~mask; - - if (0xFFFF == mask) - { - res_data.insert(data_pos, data_pos + SIMD_BYTES); - } - else - { - while (mask) - { - size_t index = __builtin_ctz(mask); - res_data.push_back(data_pos[index]); - mask = mask & (mask - 1); - } - } - - filt_pos += SIMD_BYTES; - data_pos += SIMD_BYTES; - } - -#endif - while (filt_pos < filt_end) { if (*filt_pos) diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index ea5ca4e45b4..5b9d6344aee 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -12,24 +12,39 @@ namespace DB { -#if defined(__SSE2__) && defined(__POPCNT__) -/// Transform 64-byte mask to 64-bit mask. -static UInt64 toBits64(const Int8 * bytes64) +/// Transform 64-byte mask to 64-bit mask +UInt64 Bytes64MaskToBits64Mask(const UInt8 * bytes64) { +#if defined(__AVX512F__) && defined(__AVX512BW__) + static const __m512i zero64 = _mm512_setzero_epi32(); + UInt64 res = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(bytes64)), zero64, _MM_CMPINT_GT); +#elif defined(__AVX__) && defined(__AVX2__) + static const __m256i zero32 = _mm256_setzero_si256(); + UInt64 res = + (static_cast(_mm256_movemask_epi8(_mm256_cmpgt_epi8( + _mm256_loadu_si256(reinterpret_cast(bytes64)), zero32))) & 0xffffffff) + | (static_cast(_mm256_movemask_epi8(_mm256_cmpgt_epi8( + _mm256_loadu_si256(reinterpret_cast(bytes64+32)), zero32))) << 32); +#elif defined(__SSE2__) && defined(__POPCNT__) static const __m128i zero16 = _mm_setzero_si128(); UInt64 res = - static_cast(_mm_movemask_epi8(_mm_cmpeq_epi8( - _mm_loadu_si128(reinterpret_cast(bytes64)), zero16))) - | (static_cast(_mm_movemask_epi8(_mm_cmpeq_epi8( - _mm_loadu_si128(reinterpret_cast(bytes64 + 16)), zero16))) << 16) - | (static_cast(_mm_movemask_epi8(_mm_cmpeq_epi8( - _mm_loadu_si128(reinterpret_cast(bytes64 + 32)), zero16))) << 32) - | (static_cast(_mm_movemask_epi8(_mm_cmpeq_epi8( - _mm_loadu_si128(reinterpret_cast(bytes64 + 48)), zero16))) << 48); - - return ~res; -} + static_cast(_mm_movemask_epi8(_mm_cmpgt_epi8( + _mm_loadu_si128(reinterpret_cast(bytes64)), zero16))) + | (static_cast(_mm_movemask_epi8(_mm_cmpgt_epi8( + _mm_loadu_si128(reinterpret_cast(bytes64 + 16)), zero16))) << 16) + | (static_cast(_mm_movemask_epi8(_mm_cmpgt_epi8( + _mm_loadu_si128(reinterpret_cast(bytes64 + 32)), zero16))) << 32) + | (static_cast(_mm_movemask_epi8(_mm_cmpgt_epi8( + _mm_loadu_si128(reinterpret_cast(bytes64 + 48)), zero16))) << 48); +#else + UInt64 res = 0; + const UInt8 * pos = bytes64; + const UInt8 * end = pos + 64; + for (; pos < end; ++pos) + res |= (*pos != 0)<<(pos-bytes64)); #endif + return res; +} size_t countBytesInFilter(const UInt8 * filt, size_t sz) { @@ -40,14 +55,14 @@ size_t countBytesInFilter(const UInt8 * filt, size_t sz) * It would be better to use != 0, then this does not allow SSE2. */ - const Int8 * pos = reinterpret_cast(filt); - const Int8 * end = pos + sz; + const UInt8 * pos = filt; + const UInt8 * end = pos + sz; #if defined(__SSE2__) && defined(__POPCNT__) - const Int8 * end64 = pos + sz / 64 * 64; + const UInt8 * end64 = pos + sz / 64 * 64; for (; pos < end64; pos += 64) - count += __builtin_popcountll(toBits64(pos)); + count += __builtin_popcountll(Bytes64MaskToBits64Mask(pos)); /// TODO Add duff device for tail? #endif @@ -72,15 +87,15 @@ size_t countBytesInFilterWithNull(const IColumn::Filter & filt, const UInt8 * nu * It would be better to use != 0, then this does not allow SSE2. */ - const Int8 * pos = reinterpret_cast(filt.data()); - const Int8 * pos2 = reinterpret_cast(null_map); - const Int8 * end = pos + filt.size(); + const UInt8 * pos = filt.data(); + const UInt8 * pos2 = null_map; + const UInt8 * end = pos + filt.size(); #if defined(__SSE2__) && defined(__POPCNT__) - const Int8 * end64 = pos + filt.size() / 64 * 64; + const UInt8 * end64 = pos + filt.size() / 64 * 64; for (; pos < end64; pos += 64, pos2 += 64) - count += __builtin_popcountll(toBits64(pos) & ~toBits64(pos2)); + count += __builtin_popcountll(Bytes64MaskToBits64Mask(pos) & ~Bytes64MaskToBits64Mask(pos2)); /// TODO Add duff device for tail? #endif diff --git a/src/Columns/ColumnsCommon.h b/src/Columns/ColumnsCommon.h index 25f87e90bba..baf26deb00c 100644 --- a/src/Columns/ColumnsCommon.h +++ b/src/Columns/ColumnsCommon.h @@ -15,6 +15,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +/// Transform 64-byte mask to 64-bit mask +UInt64 Bytes64MaskToBits64Mask(const UInt8 * bytes64); /// Counts how many bytes of `filt` are greater than zero. size_t countBytesInFilter(const UInt8 * filt, size_t sz); size_t countBytesInFilter(const IColumn::Filter & filt); From 3810baf44cd63ded64f886d38aea10d6a05450e0 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 26 Oct 2021 00:00:57 +0300 Subject: [PATCH 082/173] CompiledExpressionCache limit elements size --- programs/server/Server.cpp | 7 ++++++- programs/server/config.xml | 5 ++++- programs/server/config.yaml.example | 5 ++++- src/Common/LRUCache.h | 14 +++++++------- src/Interpreters/JIT/CompiledExpressionCache.cpp | 4 ++-- src/Interpreters/JIT/CompiledExpressionCache.h | 2 +- 6 files changed, 24 insertions(+), 13 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index d850ca45e26..2be9e57e437 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -957,9 +957,14 @@ if (ThreadFuzzer::instance().isEffective()) global_context->setMMappedFileCache(mmap_cache_size); #if USE_EMBEDDED_COMPILER + /// 128 MB constexpr size_t compiled_expression_cache_size_default = 1024 * 1024 * 128; size_t compiled_expression_cache_size = config().getUInt64("compiled_expression_cache_size", compiled_expression_cache_size_default); - CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_size); + + constexpr size_t compiled_expression_cache_elements_size_default = 10000; + size_t compiled_expression_cache_elements_size = config().getUInt64("compiled_expression_cache_elements_size", compiled_expression_cache_elements_size_default); + + CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_size, compiled_expression_cache_elements_size); #endif /// Set path for format schema files diff --git a/programs/server/config.xml b/programs/server/config.xml index e38a6daeaed..37f36aa5215 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -351,9 +351,12 @@ --> 1000 - + 134217728 + + 10000 + /var/lib/clickhouse/ diff --git a/programs/server/config.yaml.example b/programs/server/config.yaml.example index b2d016e06e5..c312e6a2208 100644 --- a/programs/server/config.yaml.example +++ b/programs/server/config.yaml.example @@ -279,9 +279,12 @@ mark_cache_size: 5368709120 # also it can be dropped manually by the SYSTEM DROP MMAP CACHE query. mmap_cache_size: 1000 -# Cache size for compiled expressions. +# Cache size in bytes for compiled expressions. compiled_expression_cache_size: 134217728 +# Cache size in elements for compiled expressions. +compiled_expression_cache_elements_size: 10000 + # Path to data directory, with trailing slash. path: /var/lib/clickhouse/ diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 93abfce3c4e..3df27cbdce9 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -36,12 +36,10 @@ public: using Mapped = TMapped; using MappedPtr = std::shared_ptr; -private: - using Clock = std::chrono::steady_clock; - -public: - LRUCache(size_t max_size_) - : max_size(std::max(static_cast(1), max_size_)) {} + LRUCache(size_t max_size_, size_t max_elements_size_ = 0) + : max_size(std::max(static_cast(1), max_size_)) + , max_elements_size(max_elements_size_) + {} MappedPtr get(const Key & key) { @@ -252,6 +250,7 @@ private: /// Total weight of values. size_t current_size = 0; const size_t max_size; + const size_t max_elements_size; std::atomic hits {0}; std::atomic misses {0}; @@ -311,7 +310,8 @@ private: { size_t current_weight_lost = 0; size_t queue_size = cells.size(); - while ((current_size > max_size) && (queue_size > 1)) + + while ((current_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) && (queue_size > 1)) { const Key & key = queue.front(); diff --git a/src/Interpreters/JIT/CompiledExpressionCache.cpp b/src/Interpreters/JIT/CompiledExpressionCache.cpp index 98f4eec982d..674e02236f5 100644 --- a/src/Interpreters/JIT/CompiledExpressionCache.cpp +++ b/src/Interpreters/JIT/CompiledExpressionCache.cpp @@ -16,12 +16,12 @@ CompiledExpressionCacheFactory & CompiledExpressionCacheFactory::instance() return factory; } -void CompiledExpressionCacheFactory::init(size_t cache_size) +void CompiledExpressionCacheFactory::init(size_t cache_size_in_bytes, size_t cache_size_in_elements) { if (cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "CompiledExpressionCache was already initialized"); - cache = std::make_unique(cache_size); + cache = std::make_unique(cache_size_in_bytes, cache_size_in_elements); } CompiledExpressionCache * CompiledExpressionCacheFactory::tryGetCache() diff --git a/src/Interpreters/JIT/CompiledExpressionCache.h b/src/Interpreters/JIT/CompiledExpressionCache.h index 5182a77d77a..7d20627d5d2 100644 --- a/src/Interpreters/JIT/CompiledExpressionCache.h +++ b/src/Interpreters/JIT/CompiledExpressionCache.h @@ -52,7 +52,7 @@ private: public: static CompiledExpressionCacheFactory & instance(); - void init(size_t cache_size); + void init(size_t cache_size_in_bytes, size_t cache_size_in_elements); CompiledExpressionCache * tryGetCache(); }; From 16cd70aee908b2001649650e4def185d217393fe Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 22 Oct 2021 00:12:30 +0300 Subject: [PATCH 083/173] Fix PREWHERE with WHERE in case of always true PREWHERE --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 13 +++++++++---- .../02021_prewhere_always_true_where.reference | 1 + .../02021_prewhere_always_true_where.sql | 5 +++++ 3 files changed, 15 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02021_prewhere_always_true_where.reference create mode 100644 tests/queries/0_stateless/02021_prewhere_always_true_where.sql diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 2f6bc10e472..c5454f33c79 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -1044,10 +1044,15 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r /// Filter in WHERE instead else { - auto type = getSampleBlock().getByName(prewhere_info->prewhere_column_name).type; - ColumnWithTypeAndName col(result.getFilterHolder()->convertToFullColumnIfConst(), std::make_shared(), ""); - result.columns[prewhere_column_pos] = castColumn(col, type); - result.clearFilter(); // Acting as a flag to not filter in PREWHERE + if (prewhere_info->remove_prewhere_column) + result.columns.erase(result.columns.begin() + prewhere_column_pos); + else + { + auto type = getSampleBlock().getByName(prewhere_info->prewhere_column_name).type; + ColumnWithTypeAndName col(result.getFilterHolder()->convertToFullColumnIfConst(), std::make_shared(), ""); + result.columns[prewhere_column_pos] = castColumn(col, type); + result.clearFilter(); // Acting as a flag to not filter in PREWHERE + } } } diff --git a/tests/queries/0_stateless/02021_prewhere_always_true_where.reference b/tests/queries/0_stateless/02021_prewhere_always_true_where.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02021_prewhere_always_true_where.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02021_prewhere_always_true_where.sql b/tests/queries/0_stateless/02021_prewhere_always_true_where.sql new file mode 100644 index 00000000000..95dcb6a15c2 --- /dev/null +++ b/tests/queries/0_stateless/02021_prewhere_always_true_where.sql @@ -0,0 +1,5 @@ +drop table if exists data_02021; +create table data_02021 (key Int) engine=MergeTree() order by key; +insert into data_02021 values (1); +select count() from data_02021 prewhere 1 or ignore(key) where ignore(key)=0; +drop table data_02021; From 911b9c3973fb75c130392a5c1e89e83e42f4236e Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Tue, 26 Oct 2021 06:02:57 +0530 Subject: [PATCH 084/173] Improve function defect --- src/Columns/ColumnsCommon.cpp | 119 ++++++---------------------------- 1 file changed, 20 insertions(+), 99 deletions(-) diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index 5b9d6344aee..2be9b18de4e 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -17,33 +17,33 @@ UInt64 Bytes64MaskToBits64Mask(const UInt8 * bytes64) { #if defined(__AVX512F__) && defined(__AVX512BW__) static const __m512i zero64 = _mm512_setzero_epi32(); - UInt64 res = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(bytes64)), zero64, _MM_CMPINT_GT); + UInt64 res = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(bytes64)), zero64, _MM_CMPINT_EQ); #elif defined(__AVX__) && defined(__AVX2__) static const __m256i zero32 = _mm256_setzero_si256(); UInt64 res = - (static_cast(_mm256_movemask_epi8(_mm256_cmpgt_epi8( + (static_cast(_mm256_movemask_epi8(_mm256_cmpeq_epi8( _mm256_loadu_si256(reinterpret_cast(bytes64)), zero32))) & 0xffffffff) - | (static_cast(_mm256_movemask_epi8(_mm256_cmpgt_epi8( + | (static_cast(_mm256_movemask_epi8(_mm256_cmpeq_epi8( _mm256_loadu_si256(reinterpret_cast(bytes64+32)), zero32))) << 32); #elif defined(__SSE2__) && defined(__POPCNT__) static const __m128i zero16 = _mm_setzero_si128(); UInt64 res = - static_cast(_mm_movemask_epi8(_mm_cmpgt_epi8( - _mm_loadu_si128(reinterpret_cast(bytes64)), zero16))) - | (static_cast(_mm_movemask_epi8(_mm_cmpgt_epi8( - _mm_loadu_si128(reinterpret_cast(bytes64 + 16)), zero16))) << 16) - | (static_cast(_mm_movemask_epi8(_mm_cmpgt_epi8( - _mm_loadu_si128(reinterpret_cast(bytes64 + 32)), zero16))) << 32) - | (static_cast(_mm_movemask_epi8(_mm_cmpgt_epi8( - _mm_loadu_si128(reinterpret_cast(bytes64 + 48)), zero16))) << 48); + (static_cast(_mm_movemask_epi8(_mm_cmpeq_epi8( + _mm_loadu_si128(reinterpret_cast(bytes64)), zero16))) & 0xffff) + | ((static_cast(_mm_movemask_epi8(_mm_cmpeq_epi8( + _mm_loadu_si128(reinterpret_cast(bytes64 + 16)), zero16))) << 16) & 0xffff0000) + | ((static_cast(_mm_movemask_epi8(_mm_cmpeq_epi8( + _mm_loadu_si128(reinterpret_cast(bytes64 + 32)), zero16))) << 32) & 0xffff00000000) + | ((static_cast(_mm_movemask_epi8(_mm_cmpeq_epi8( + _mm_loadu_si128(reinterpret_cast(bytes64 + 48)), zero16))) << 48) & 0xffff000000000000); #else UInt64 res = 0; const UInt8 * pos = bytes64; const UInt8 * end = pos + 64; for (; pos < end; ++pos) - res |= (*pos != 0)<<(pos-bytes64)); + res |= (*pos == 0)<<(pos-bytes64)); #endif - return res; + return ~res; } size_t countBytesInFilter(const UInt8 * filt, size_t sz) @@ -244,16 +244,19 @@ namespace memcpy(&res_elems[elems_size_old], &src_elems[arr_offset], arr_size * sizeof(T)); }; - #if defined(__AVX512F__) && defined(__AVX512BW__) - const __m512i zero_vec = _mm512_setzero_epi32(); + /** A slightly more optimized version. + * Based on the assumption that often pieces of consecutive values + * completely pass or do not pass the filter. + * Therefore, we will optimistically check the parts of `SIMD_BYTES` values. + */ static constexpr size_t SIMD_BYTES = 64; const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; while (filt_pos < filt_end_aligned) { - uint64_t mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero_vec, _MM_CMPINT_GT); + uint64_t mask = Bytes64MaskToBits64Mask(filt_pos); - if (mask == 0xffffffffffffffff) + if (0xffffffffffffffff == mask) { /// SIMD_BYTES consecutive rows pass the filter const auto first = offsets_pos == offsets_begin; @@ -285,88 +288,6 @@ namespace filt_pos += SIMD_BYTES; offsets_pos += SIMD_BYTES; } - #elif defined(__AVX__) && defined(__AVX2__) - const __m256i zero_vec = _mm256_setzero_si256(); - static constexpr size_t SIMD_BYTES = 32; - const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; - - while (filt_pos < filt_end_aligned) - { - uint32_t mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero_vec)); - - if (mask == 0xffffffff) - { - /// SIMD_BYTES consecutive rows pass the filter - const auto first = offsets_pos == offsets_begin; - - const auto chunk_offset = first ? 0 : offsets_pos[-1]; - const auto chunk_size = offsets_pos[SIMD_BYTES - 1] - chunk_offset; - - result_offsets_builder.template insertChunk(offsets_pos, first, chunk_offset, chunk_size); - - /// copy elements for SIMD_BYTES arrays at once - const auto elems_size_old = res_elems.size(); - res_elems.resize(elems_size_old + chunk_size); - memcpy(&res_elems[elems_size_old], &src_elems[chunk_offset], chunk_size * sizeof(T)); - } - else - { - while (mask) - { - size_t index = __builtin_ctz(mask); - copy_array(offsets_pos + index); - #ifdef __BMI__ - mask = _blsr_u32(mask); - #else - mask = mask & (mask-1); - #endif - } - } - - filt_pos += SIMD_BYTES; - offsets_pos += SIMD_BYTES; - } - #elif defined(__SSE2__) - const __m128i zero_vec = _mm_setzero_si128(); - static constexpr size_t SIMD_BYTES = 16; - const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; - - while (filt_pos < filt_end_aligned) - { - UInt16 mask = _mm_movemask_epi8(_mm_cmpeq_epi8( - _mm_loadu_si128(reinterpret_cast(filt_pos)), - zero_vec)); - mask = ~mask; - - if (mask == 0xffff) - { - /// SIMD_BYTES consecutive rows pass the filter - const auto first = offsets_pos == offsets_begin; - - const auto chunk_offset = first ? 0 : offsets_pos[-1]; - const auto chunk_size = offsets_pos[SIMD_BYTES - 1] - chunk_offset; - - result_offsets_builder.template insertChunk(offsets_pos, first, chunk_offset, chunk_size); - - /// copy elements for SIMD_BYTES arrays at once - const auto elems_size_old = res_elems.size(); - res_elems.resize(elems_size_old + chunk_size); - memcpy(&res_elems[elems_size_old], &src_elems[chunk_offset], chunk_size * sizeof(T)); - } - else - { - while (mask) - { - size_t index = __builtin_ctz(mask); - copy_array(offsets_pos + index); - mask = mask & (mask - 1); - } - } - - filt_pos += SIMD_BYTES; - offsets_pos += SIMD_BYTES; - } - #endif while (filt_pos < filt_end) { From 6e3921f77e3e52e6b9b15e6d861a2f1e7924a13f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Tue, 26 Oct 2021 09:12:13 +0800 Subject: [PATCH 085/173] delete symbolic links --- .../external-authenticators/kerberos.md | 105 ------------------ 1 file changed, 105 deletions(-) delete mode 120000 docs/zh/operations/external-authenticators/kerberos.md diff --git a/docs/zh/operations/external-authenticators/kerberos.md b/docs/zh/operations/external-authenticators/kerberos.md deleted file mode 120000 index 1121a86a793..00000000000 --- a/docs/zh/operations/external-authenticators/kerberos.md +++ /dev/null @@ -1,105 +0,0 @@ -# Kerberos认证 {#external-authenticators-kerberos} -现有正确配置的 ClickHouse 用户可以通过 Kerberos 身份验证协议进行身份验证. - -目前, Kerberos 只能用作现有用户的外部身份验证器,这些用户在 `users.xml` 或本地访问控制路径中定义. -这些用户只能使用 HTTP 请求, 并且必须能够使用 GSS-SPNEGO 机制进行身份验证. - -对于这种方法, 必须在系统中配置 Kerberos, 且必须在 ClickHouse 配置中启用. - -## 开启Kerberos {#enabling-kerberos-in-clickHouse} -要启用 Kerberos, 应该在 `config.xml` 中包含 `kerberos` 部分. 此部分可能包含其他参数. - -#### 参数: {#parameters} -- `principal` - 将在接受安全上下文时获取和使用的规范服务主体名称. -- 此参数是可选的, 如果省略, 将使用默认主体. - -- `realm` - 一个领域, 用于将身份验证限制为仅那些发起者领域与其匹配的请求. - -- 此参数是可选的,如果省略,则不会应用其他领域的过滤. - -示例 (进入 `config.xml`): -```xml - - - - -``` - -主体规范: -```xml - - - - HTTP/clickhouse.example.com@EXAMPLE.COM - - -``` - -按领域过滤: -```xml - - - - EXAMPLE.COM - - -``` - -!!! warning "注意" - -您只能定义一个 `kerberos` 部分. 多个 `kerberos` 部分的存在将强制 ClickHouse 禁用 Kerberos 身份验证. - -!!! warning "注意" - -`主体`和`领域`部分不能同时指定. `主体`和`领域`的出现将迫使ClickHouse禁用Kerberos身份验证. - -## Kerberos作为现有用户的外部身份验证器 {#kerberos-as-an-external-authenticator-for-existing-users} -Kerberos可以用作验证本地定义用户(在`users.xml`或本地访问控制路径中定义的用户)身份的方法。目前,**只有**通过HTTP接口的请求才能被认证(通过GSS-SPNEGO机制). - -Kerberos主体名称格式通常遵循以下模式: -- *primary/instance@REALM* - -*/instance* 部分可能出现零次或多次. **发起者的规范主体名称的主要部分应与被认证用户名匹配, 以便身份验证成功**. - -### `users.xml`中启用Kerberos {#enabling-kerberos-in-users-xml} -为了启用用户的 Kerberos 身份验证, 请在用户定义中指定 `kerberos` 部分而不是`密码`或类似部分. - -参数: -- `realm` - 用于将身份验证限制为仅那些发起者的领域与其匹配的请求的领域. -- 此参数是可选的, 如果省略, 则不会应用其他按领域的过滤. - -示例 (进入 `users.xml`): -``` - - - - - - - - EXAMPLE.COM - - - - -``` - -!!! warning "警告" - -注意, Kerberos身份验证不能与任何其他身份验证机制一起使用. 任何其他部分(如`密码`和`kerberos`)的出现都会迫使ClickHouse关闭. - -!!! info "提醒" - -请注意, 现在, 一旦用户 `my_user` 使用 `kerberos`, 必须在主 `config.xml` 文件中启用 Kerberos,如前所述. - -### 使用 SQL 启用 Kerberos {#enabling-kerberos-using-sql} -在 ClickHouse 中启用 [SQL 驱动的访问控制和帐户管理](./access-rights.md#access-control)后, 也可以使用 SQL 语句创建由 Kerberos 识别的用户. - -```sql -CREATE USER my_user IDENTIFIED WITH kerberos REALM 'EXAMPLE.COM' -``` - -...或者, 不按领域过滤: -```sql -CREATE USER my_user IDENTIFIED WITH kerberos -``` From 37d7eb0cb82fa47b453795fcffdef934d9c4791c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Tue, 26 Oct 2021 09:13:26 +0800 Subject: [PATCH 086/173] replace symbolic links --- .../external-authenticators/kerberos | 105 ++++++++++++++++++ 1 file changed, 105 insertions(+) create mode 100644 docs/zh/operations/external-authenticators/kerberos diff --git a/docs/zh/operations/external-authenticators/kerberos b/docs/zh/operations/external-authenticators/kerberos new file mode 100644 index 00000000000..1121a86a793 --- /dev/null +++ b/docs/zh/operations/external-authenticators/kerberos @@ -0,0 +1,105 @@ +# Kerberos认证 {#external-authenticators-kerberos} +现有正确配置的 ClickHouse 用户可以通过 Kerberos 身份验证协议进行身份验证. + +目前, Kerberos 只能用作现有用户的外部身份验证器,这些用户在 `users.xml` 或本地访问控制路径中定义. +这些用户只能使用 HTTP 请求, 并且必须能够使用 GSS-SPNEGO 机制进行身份验证. + +对于这种方法, 必须在系统中配置 Kerberos, 且必须在 ClickHouse 配置中启用. + +## 开启Kerberos {#enabling-kerberos-in-clickHouse} +要启用 Kerberos, 应该在 `config.xml` 中包含 `kerberos` 部分. 此部分可能包含其他参数. + +#### 参数: {#parameters} +- `principal` - 将在接受安全上下文时获取和使用的规范服务主体名称. +- 此参数是可选的, 如果省略, 将使用默认主体. + +- `realm` - 一个领域, 用于将身份验证限制为仅那些发起者领域与其匹配的请求. + +- 此参数是可选的,如果省略,则不会应用其他领域的过滤. + +示例 (进入 `config.xml`): +```xml + + + + +``` + +主体规范: +```xml + + + + HTTP/clickhouse.example.com@EXAMPLE.COM + + +``` + +按领域过滤: +```xml + + + + EXAMPLE.COM + + +``` + +!!! warning "注意" + +您只能定义一个 `kerberos` 部分. 多个 `kerberos` 部分的存在将强制 ClickHouse 禁用 Kerberos 身份验证. + +!!! warning "注意" + +`主体`和`领域`部分不能同时指定. `主体`和`领域`的出现将迫使ClickHouse禁用Kerberos身份验证. + +## Kerberos作为现有用户的外部身份验证器 {#kerberos-as-an-external-authenticator-for-existing-users} +Kerberos可以用作验证本地定义用户(在`users.xml`或本地访问控制路径中定义的用户)身份的方法。目前,**只有**通过HTTP接口的请求才能被认证(通过GSS-SPNEGO机制). + +Kerberos主体名称格式通常遵循以下模式: +- *primary/instance@REALM* + +*/instance* 部分可能出现零次或多次. **发起者的规范主体名称的主要部分应与被认证用户名匹配, 以便身份验证成功**. + +### `users.xml`中启用Kerberos {#enabling-kerberos-in-users-xml} +为了启用用户的 Kerberos 身份验证, 请在用户定义中指定 `kerberos` 部分而不是`密码`或类似部分. + +参数: +- `realm` - 用于将身份验证限制为仅那些发起者的领域与其匹配的请求的领域. +- 此参数是可选的, 如果省略, 则不会应用其他按领域的过滤. + +示例 (进入 `users.xml`): +``` + + + + + + + + EXAMPLE.COM + + + + +``` + +!!! warning "警告" + +注意, Kerberos身份验证不能与任何其他身份验证机制一起使用. 任何其他部分(如`密码`和`kerberos`)的出现都会迫使ClickHouse关闭. + +!!! info "提醒" + +请注意, 现在, 一旦用户 `my_user` 使用 `kerberos`, 必须在主 `config.xml` 文件中启用 Kerberos,如前所述. + +### 使用 SQL 启用 Kerberos {#enabling-kerberos-using-sql} +在 ClickHouse 中启用 [SQL 驱动的访问控制和帐户管理](./access-rights.md#access-control)后, 也可以使用 SQL 语句创建由 Kerberos 识别的用户. + +```sql +CREATE USER my_user IDENTIFIED WITH kerberos REALM 'EXAMPLE.COM' +``` + +...或者, 不按领域过滤: +```sql +CREATE USER my_user IDENTIFIED WITH kerberos +``` From 8ec04169b68039d51243aa6e117dc4bc5b4d633e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Tue, 26 Oct 2021 09:14:17 +0800 Subject: [PATCH 087/173] replace symbolic links --- .../operations/external-authenticators/{kerberos => kerberos.md} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename docs/zh/operations/external-authenticators/{kerberos => kerberos.md} (100%) diff --git a/docs/zh/operations/external-authenticators/kerberos b/docs/zh/operations/external-authenticators/kerberos.md similarity index 100% rename from docs/zh/operations/external-authenticators/kerberos rename to docs/zh/operations/external-authenticators/kerberos.md From 5dc6c7a157f308bbee6f5f6fd056bf4ceb28e24a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Tue, 26 Oct 2021 09:32:06 +0800 Subject: [PATCH 088/173] adjust title level --- docs/zh/development/continuous-integration.md | 52 +++++++++---------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/docs/zh/development/continuous-integration.md b/docs/zh/development/continuous-integration.md index 7e4c1bf36a9..41e22074915 120000 --- a/docs/zh/development/continuous-integration.md +++ b/docs/zh/development/continuous-integration.md @@ -17,50 +17,50 @@ git push 如果您不确定要做什么,可以向维护人员寻求帮助. -# 与Master合并 {#merge-with-master} +## 与Master合并 {#merge-with-master} 验证PR是否可以合并到master. 如果没有, 它将失败并显示消息'Cannot fetch mergecommit'的.请按[GitHub文档](https://docs.github.com/en/github/collaborating-with-pull-requests/addressing-merge-conflicts/resolving-a-merge-conflict-on-github)中描述的冲突解决, 或使用git将主分支合并到您的pull请求分支来修复这个检查. -# 文档检查 {#docs-check} +## 文档检查 {#docs-check} 尝试构建ClickHouse文档网站. 如果您更改了文档中的某些内容, 它可能会失败. 最可能的原因是文档中的某些交叉链接是错误的. 转到检查报告并查找`ERROR`和`WARNING`消息. -# 报告详情 {#report-details} +### 报告详情 {#report-details} - [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) - `docs_output.txt`包含构建日志信息. [成功结果案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check/docs_output.txt) -# 描述信息检查 {#description-check} +## 描述信息检查 {#description-check} 检查pull请求的描述是否符合[PULL_REQUEST_TEMPLATE.md](../../../.github/PULL_REQUEST_TEMPLATE.md)模板. 您必须为您的更改指定一个更改日志类别(例如,Bug修复), 并且为[CHANGELOG.md](../whats-new/changelog/)编写一条用户可读的消息用来描述更改. -# 推送到DockerHub {#push-to-dockerhub} +## 推送到DockerHub {#push-to-dockerhub} 生成用于构建和测试的docker映像, 然后将它们推送到DockerHub. -# 标记检查 {#marker-check} +## 标记检查 {#marker-check} 该检查意味着CI系统已经开始处理PR.当它处于'待处理'状态时,意味着尚未开始所有检查. 启动所有检查后,状态更改为'成功'. # 格式检查 {#style-check} 使用`utils/check-style/check-style`二进制文件执行一些简单的基于正则表达式的代码样式检查(注意, 它可以在本地运行). 如果失败, 按照[代码样式指南](./style.md)修复样式错误. -# 报告详情 {#report-details} +### 报告详情 {#report-details} - [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check.html) - `docs_output.txt`记录了查结果错误(无效表格等), 空白页表示没有错误. [成功结果案例](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check/output.txt) -# PVS 检查 {#pvs-check} +### PVS 检查 {#pvs-check} 使用静态分析工具[PVS-studio](https://www.viva64.com/en/pvs-studio/)检查代码. 查看报告以查看确切的错误.如果可以则修复它们, 如果不行, 可以向ClickHouse的维护人员寻求帮忙. -# 报告详情 {#report-details} +### 报告详情 {#report-details} - [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/pvs_check.html) - `test_run.txt.out.log`包含构建和分析日志文件.它只包含解析或未找到的错误. - `HTML report`包含分析结果.有关说明请访问PVS的[官方网站](https://www.viva64.com/en/m/0036/#ID14E9A2B2CD) -# 快速测试 {#fast-test} +## 快速测试 {#fast-test} 通常情况下这是PR运行的第一个检查.它构建ClickHouse以及大多数无状态运行测试, 其中省略了一些.如果失败,在修复之前不会开始进一步的检查. 查看报告以了解哪些测试失败, 然后按照[此处](./tests.md#functional-test-locally)描述的在本地重现失败. -# 报告详情 {#report-details} +### 报告详情 {#report-details} [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/fast_test.html) -# 状态页文件 {#status-page-files} +#### 状态页文件 {#status-page-files} - `runlog.out.log` 是包含所有其他日志的通用日志. - `test_log.txt` - `submodule_log.txt` 包含关于克隆和检查所需子模块的消息. @@ -73,15 +73,15 @@ git push - `build_log.txt` - `cmake_log.txt` 包含关于C/C++和Linux标志检查的消息. -# 状态页列信息 {#status-page-columns} +#### 状态页列信息 {#status-page-columns} - 测试名称 -- 包含测试的名称(不带路径, 例如, 所有类型的测试将被剥离到该名称). - 测试状态 -- 跳过、成功或失败之一. - 测试时间, 秒. -- 这个测试是空的. -# 建构检查 {#build-check} +## 建构检查 {#build-check} 在各种配置中构建ClickHouse, 以便在后续步骤中使用. 您必须修复失败的构建.构建日志通常有足够的信息来修复错误, 但是您可能必须在本地重现故障. `cmake`选项可以在构建日志中通过grep `cmake`操作找到.使用这些选项并遵循[一般的构建过程](./build.md). -# 报告详情 {#report-details} +### 报告详情 {#report-details} [状态页示例](https://clickhouse-builds.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/clickhouse_build_check/report.html) - **Compiler**: `gcc-9` 或 `clang-10` (或其他架构的`clang-10-xx`, 比如`clang-10-freebsd`). - **Build type**: `Debug` or `RelWithDebInfo` (cmake). @@ -105,22 +105,22 @@ git push - `shared_build.tgz`: 使用共享库构建. - `performance.tgz`: 用于性能测试的特殊包. -# 特殊构建检查 {#special-buildcheck} +## 特殊构建检查 {#special-buildcheck} 使用clang-tidy执行静态分析和代码样式检查. 该报告类似于构建检查. 修复在构建日志中发现的错误. -# 功能无状态测试 {#functional-stateless-tests} +## 功能无状态测试 {#functional-stateless-tests} 为构建在不同配置中的ClickHouse二进制文件运行[无状态功能测试](./tests.md#functional-tests)——发布、调试、使用杀毒软件等.通过报告查看哪些测试失败,然后按照[此处](./tests.md#functional-test-locally)描述的在本地重现失败.注意, 您必须使用正确的构建配置来重现——在AddressSanitizer下测试可能失败,但在Debug中可以通过.从[CI构建检查页面](./build.md#you-dont-have-to-build-clickhouse)下载二进制文件, 或者在本地构建它. -# 功能有状态测试 {#functional-stateful-tests} +## 功能有状态测试 {#functional-stateful-tests} 运行[有状态功能测试](./tests.md#functional-tests).以无状态功能测试相同的方式对待它们.不同之处在于它们需要从[Yandex.Metrica数据集](https://clickhouse.com/docs/en/getting-started/example-datasets/metrica/)的`hits`和`visits`表来运行. -# 集成测试 {#integration-tests} +## 集成测试 {#integration-tests} 运行[集成测试](./tests.md#integration-tests). -# Testflows 检查{#testflows-check} +## Testflows 检查{#testflows-check} 使用Testflows测试系统去运行一些测试, 在[此处](../../../tests/testflows/README.md#running-tests-locally)查看如何在本地运行它们. -# 压力测试 {#stress-test} +## 压力测试 {#stress-test} 从多个客户端并发运行无状态功能测试, 用以检测与并发相关的错误.如果失败: ``` * Fix all other test failures first; @@ -128,7 +128,7 @@ git push of error. ``` -# 冒烟测试 {#split-build-smoke-test} +## 冒烟测试 {#split-build-smoke-test} 检查[拆分构建](./build.md#split-build)配置中的服务器构建是否可以启动并运行简单查询.如果失败: ``` * Fix other test errors first; @@ -136,16 +136,16 @@ git push locally and check whether it can start and run `select 1`. ``` -# 兼容性检查 {#compatibility-check} +## 兼容性检查 {#compatibility-check} 检查`clickhouse`二进制文件是否可以在带有旧libc版本的发行版上运行.如果失败, 请向维护人员寻求帮助. -# AST模糊器 {#ast-fuzzer} +## AST模糊器 {#ast-fuzzer} 运行随机生成的查询来捕获程序错误.如果失败, 请向维护人员寻求帮助. -# 性能测试 {#performance-tests} +## 性能测试 {#performance-tests} 测量查询性能的变化. 这是最长的检查, 只需不到 6 小时即可运行.性能测试报告在[此处](../../../docker/test/performance-comparison/README.md#how-to-read-the-report)有详细描述. -# 质量保证 {#qa} +## 质量保证 {#qa} 什么是状态页面上的任务(专用网络)项目? 它是 Yandex 内部工作系统的链接. Yandex 员工可以看到检查的开始时间及其更详细的状态. From aa4312760128c70cb606d364ac8ce545c5a06c49 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Tue, 26 Oct 2021 09:33:16 +0800 Subject: [PATCH 089/173] delete symbolic links --- docs/zh/development/continuous-integration.md | 155 ------------------ 1 file changed, 155 deletions(-) delete mode 120000 docs/zh/development/continuous-integration.md diff --git a/docs/zh/development/continuous-integration.md b/docs/zh/development/continuous-integration.md deleted file mode 120000 index 41e22074915..00000000000 --- a/docs/zh/development/continuous-integration.md +++ /dev/null @@ -1,155 +0,0 @@ -# 持续集成检查 {#continuous-integration-checks} -当你提交一个pull请求时, ClickHouse[持续集成(CI)系统](../tests.md#test-automation)会对您的代码运行一些自动检查. - -这在存储库维护者(来自ClickHouse团队的人)筛选了您的代码并将可测试标签添加到您的pull请求之后发生. - -检查的结果被列在[GitHub检查文档](https://docs.github.com/en/github/collaborating-with-pull-requests/collaborating-on-repositories-with-code-quality-features/about-status-checks)中所述的GitHub pull请求页面. - -如果检查失败,您可能被要求去修复它. 该界面介绍了您可能遇到的检查,以及如何修复它们. - -如果检查失败看起来与您的更改无关, 那么它可能是一些暂时的故障或基础设施问题. 向pull请求推一个空的commit以重新启动CI检查: - -``` -git reset -git commit --allow-empty -git push -``` - -如果您不确定要做什么,可以向维护人员寻求帮助. - -## 与Master合并 {#merge-with-master} -验证PR是否可以合并到master. 如果没有, 它将失败并显示消息'Cannot fetch mergecommit'的.请按[GitHub文档](https://docs.github.com/en/github/collaborating-with-pull-requests/addressing-merge-conflicts/resolving-a-merge-conflict-on-github)中描述的冲突解决, 或使用git将主分支合并到您的pull请求分支来修复这个检查. - -## 文档检查 {#docs-check} -尝试构建ClickHouse文档网站. 如果您更改了文档中的某些内容, 它可能会失败. 最可能的原因是文档中的某些交叉链接是错误的. 转到检查报告并查找`ERROR`和`WARNING`消息. - -### 报告详情 {#report-details} -- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) -- `docs_output.txt`包含构建日志信息. [成功结果案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check/docs_output.txt) - -## 描述信息检查 {#description-check} -检查pull请求的描述是否符合[PULL_REQUEST_TEMPLATE.md](../../../.github/PULL_REQUEST_TEMPLATE.md)模板. - -您必须为您的更改指定一个更改日志类别(例如,Bug修复), 并且为[CHANGELOG.md](../whats-new/changelog/)编写一条用户可读的消息用来描述更改. - -## 推送到DockerHub {#push-to-dockerhub} -生成用于构建和测试的docker映像, 然后将它们推送到DockerHub. - -## 标记检查 {#marker-check} -该检查意味着CI系统已经开始处理PR.当它处于'待处理'状态时,意味着尚未开始所有检查. 启动所有检查后,状态更改为'成功'. - -# 格式检查 {#style-check} -使用`utils/check-style/check-style`二进制文件执行一些简单的基于正则表达式的代码样式检查(注意, 它可以在本地运行). -如果失败, 按照[代码样式指南](./style.md)修复样式错误. - -### 报告详情 {#report-details} -- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check.html) -- `docs_output.txt`记录了查结果错误(无效表格等), 空白页表示没有错误. [成功结果案例](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check/output.txt) - -### PVS 检查 {#pvs-check} -使用静态分析工具[PVS-studio](https://www.viva64.com/en/pvs-studio/)检查代码. 查看报告以查看确切的错误.如果可以则修复它们, 如果不行, 可以向ClickHouse的维护人员寻求帮忙. - -### 报告详情 {#report-details} -- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/pvs_check.html) -- `test_run.txt.out.log`包含构建和分析日志文件.它只包含解析或未找到的错误. -- `HTML report`包含分析结果.有关说明请访问PVS的[官方网站](https://www.viva64.com/en/m/0036/#ID14E9A2B2CD) - -## 快速测试 {#fast-test} -通常情况下这是PR运行的第一个检查.它构建ClickHouse以及大多数无状态运行测试, 其中省略了一些.如果失败,在修复之前不会开始进一步的检查. 查看报告以了解哪些测试失败, 然后按照[此处](./tests.md#functional-test-locally)描述的在本地重现失败. - -### 报告详情 {#report-details} -[状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/fast_test.html) - -#### 状态页文件 {#status-page-files} -- `runlog.out.log` 是包含所有其他日志的通用日志. -- `test_log.txt` -- `submodule_log.txt` 包含关于克隆和检查所需子模块的消息. -- `stderr.log` -- `stdout.log` -- `clickhouse-server.log` -- `clone_log.txt` -- `install_log.txt` -- `clickhouse-server.err.log` -- `build_log.txt` -- `cmake_log.txt` 包含关于C/C++和Linux标志检查的消息. - -#### 状态页列信息 {#status-page-columns} -- 测试名称 -- 包含测试的名称(不带路径, 例如, 所有类型的测试将被剥离到该名称). -- 测试状态 -- 跳过、成功或失败之一. -- 测试时间, 秒. -- 这个测试是空的. - -## 建构检查 {#build-check} -在各种配置中构建ClickHouse, 以便在后续步骤中使用. 您必须修复失败的构建.构建日志通常有足够的信息来修复错误, 但是您可能必须在本地重现故障. `cmake`选项可以在构建日志中通过grep `cmake`操作找到.使用这些选项并遵循[一般的构建过程](./build.md). - -### 报告详情 {#report-details} -[状态页示例](https://clickhouse-builds.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/clickhouse_build_check/report.html) -- **Compiler**: `gcc-9` 或 `clang-10` (或其他架构的`clang-10-xx`, 比如`clang-10-freebsd`). -- **Build type**: `Debug` or `RelWithDebInfo` (cmake). -- **Sanitizer**: `none` (without sanitizers), `address` (ASan), `memory` (MSan), `undefined` (UBSan), or `thread` (TSan). -- **Bundled**: `bundled` 构建使用来自 `contrib` 库, 而 `unbundled` 构建使用系统库. -- **Splitted**: `splitted` is a [split build](https://clickhouse.com/docs/en/development/build/#split-build) -- **Status**: `成功` 或 `失败` -- **Build log**: 链接到构建和文件复制日志, 当构建失败时很有用. -- **Build time**. -- **Artifacts**: 构建结果文件 (`XXX`是服务器版本, 比如`20.8.1.4344`). - - `clickhouse-client_XXX_all.deb` - -` clickhouse-common-static-dbg_XXX[+asan, +msan, +ubsan, +tsan]_amd64.deb` - - `clickhouse-common-staticXXX_amd64.deb` - - `clickhouse-server_XXX_all.deb` - - `clickhouse-test_XXX_all.deb` - - `clickhouse_XXX_amd64.buildinfo` - - `clickhouse_XXX_amd64.changes` - - `clickhouse`: Main built binary. - - `clickhouse-odbc-bridge` - - `unit_tests_dbms`: 带有 ClickHouse 单元测试的 GoogleTest 二进制文件. - - `shared_build.tgz`: 使用共享库构建. - - `performance.tgz`: 用于性能测试的特殊包. - -## 特殊构建检查 {#special-buildcheck} -使用clang-tidy执行静态分析和代码样式检查. 该报告类似于构建检查. 修复在构建日志中发现的错误. - -## 功能无状态测试 {#functional-stateless-tests} -为构建在不同配置中的ClickHouse二进制文件运行[无状态功能测试](./tests.md#functional-tests)——发布、调试、使用杀毒软件等.通过报告查看哪些测试失败,然后按照[此处](./tests.md#functional-test-locally)描述的在本地重现失败.注意, 您必须使用正确的构建配置来重现——在AddressSanitizer下测试可能失败,但在Debug中可以通过.从[CI构建检查页面](./build.md#you-dont-have-to-build-clickhouse)下载二进制文件, 或者在本地构建它. - -## 功能有状态测试 {#functional-stateful-tests} -运行[有状态功能测试](./tests.md#functional-tests).以无状态功能测试相同的方式对待它们.不同之处在于它们需要从[Yandex.Metrica数据集](https://clickhouse.com/docs/en/getting-started/example-datasets/metrica/)的`hits`和`visits`表来运行. - -## 集成测试 {#integration-tests} -运行[集成测试](./tests.md#integration-tests). - -## Testflows 检查{#testflows-check} -使用Testflows测试系统去运行一些测试, 在[此处](../../../tests/testflows/README.md#running-tests-locally)查看如何在本地运行它们. - -## 压力测试 {#stress-test} -从多个客户端并发运行无状态功能测试, 用以检测与并发相关的错误.如果失败: -``` -* Fix all other test failures first; -* Look at the report to find the server logs and check them for possible causes - of error. -``` - -## 冒烟测试 {#split-build-smoke-test} -检查[拆分构建](./build.md#split-build)配置中的服务器构建是否可以启动并运行简单查询.如果失败: -``` -* Fix other test errors first; -* Build the server in [split build](./build.md#split-build) configuration - locally and check whether it can start and run `select 1`. -``` - -## 兼容性检查 {#compatibility-check} -检查`clickhouse`二进制文件是否可以在带有旧libc版本的发行版上运行.如果失败, 请向维护人员寻求帮助. - -## AST模糊器 {#ast-fuzzer} -运行随机生成的查询来捕获程序错误.如果失败, 请向维护人员寻求帮助. - -## 性能测试 {#performance-tests} -测量查询性能的变化. 这是最长的检查, 只需不到 6 小时即可运行.性能测试报告在[此处](../../../docker/test/performance-comparison/README.md#how-to-read-the-report)有详细描述. - -## 质量保证 {#qa} -什么是状态页面上的任务(专用网络)项目? - -它是 Yandex 内部工作系统的链接. Yandex 员工可以看到检查的开始时间及其更详细的状态. - -运行测试的地方 - -Yandex 内部基础设施的某个地方. From 6abe479dd8878432ea270ed9689eaa90fd39cb28 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Tue, 26 Oct 2021 09:33:53 +0800 Subject: [PATCH 090/173] replace symbolic link content. --- docs/zh/development/continuous-integration.md | 155 ++++++++++++++++++ 1 file changed, 155 insertions(+) create mode 100644 docs/zh/development/continuous-integration.md diff --git a/docs/zh/development/continuous-integration.md b/docs/zh/development/continuous-integration.md new file mode 100644 index 00000000000..41e22074915 --- /dev/null +++ b/docs/zh/development/continuous-integration.md @@ -0,0 +1,155 @@ +# 持续集成检查 {#continuous-integration-checks} +当你提交一个pull请求时, ClickHouse[持续集成(CI)系统](../tests.md#test-automation)会对您的代码运行一些自动检查. + +这在存储库维护者(来自ClickHouse团队的人)筛选了您的代码并将可测试标签添加到您的pull请求之后发生. + +检查的结果被列在[GitHub检查文档](https://docs.github.com/en/github/collaborating-with-pull-requests/collaborating-on-repositories-with-code-quality-features/about-status-checks)中所述的GitHub pull请求页面. + +如果检查失败,您可能被要求去修复它. 该界面介绍了您可能遇到的检查,以及如何修复它们. + +如果检查失败看起来与您的更改无关, 那么它可能是一些暂时的故障或基础设施问题. 向pull请求推一个空的commit以重新启动CI检查: + +``` +git reset +git commit --allow-empty +git push +``` + +如果您不确定要做什么,可以向维护人员寻求帮助. + +## 与Master合并 {#merge-with-master} +验证PR是否可以合并到master. 如果没有, 它将失败并显示消息'Cannot fetch mergecommit'的.请按[GitHub文档](https://docs.github.com/en/github/collaborating-with-pull-requests/addressing-merge-conflicts/resolving-a-merge-conflict-on-github)中描述的冲突解决, 或使用git将主分支合并到您的pull请求分支来修复这个检查. + +## 文档检查 {#docs-check} +尝试构建ClickHouse文档网站. 如果您更改了文档中的某些内容, 它可能会失败. 最可能的原因是文档中的某些交叉链接是错误的. 转到检查报告并查找`ERROR`和`WARNING`消息. + +### 报告详情 {#report-details} +- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) +- `docs_output.txt`包含构建日志信息. [成功结果案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check/docs_output.txt) + +## 描述信息检查 {#description-check} +检查pull请求的描述是否符合[PULL_REQUEST_TEMPLATE.md](../../../.github/PULL_REQUEST_TEMPLATE.md)模板. + +您必须为您的更改指定一个更改日志类别(例如,Bug修复), 并且为[CHANGELOG.md](../whats-new/changelog/)编写一条用户可读的消息用来描述更改. + +## 推送到DockerHub {#push-to-dockerhub} +生成用于构建和测试的docker映像, 然后将它们推送到DockerHub. + +## 标记检查 {#marker-check} +该检查意味着CI系统已经开始处理PR.当它处于'待处理'状态时,意味着尚未开始所有检查. 启动所有检查后,状态更改为'成功'. + +# 格式检查 {#style-check} +使用`utils/check-style/check-style`二进制文件执行一些简单的基于正则表达式的代码样式检查(注意, 它可以在本地运行). +如果失败, 按照[代码样式指南](./style.md)修复样式错误. + +### 报告详情 {#report-details} +- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check.html) +- `docs_output.txt`记录了查结果错误(无效表格等), 空白页表示没有错误. [成功结果案例](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check/output.txt) + +### PVS 检查 {#pvs-check} +使用静态分析工具[PVS-studio](https://www.viva64.com/en/pvs-studio/)检查代码. 查看报告以查看确切的错误.如果可以则修复它们, 如果不行, 可以向ClickHouse的维护人员寻求帮忙. + +### 报告详情 {#report-details} +- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/pvs_check.html) +- `test_run.txt.out.log`包含构建和分析日志文件.它只包含解析或未找到的错误. +- `HTML report`包含分析结果.有关说明请访问PVS的[官方网站](https://www.viva64.com/en/m/0036/#ID14E9A2B2CD) + +## 快速测试 {#fast-test} +通常情况下这是PR运行的第一个检查.它构建ClickHouse以及大多数无状态运行测试, 其中省略了一些.如果失败,在修复之前不会开始进一步的检查. 查看报告以了解哪些测试失败, 然后按照[此处](./tests.md#functional-test-locally)描述的在本地重现失败. + +### 报告详情 {#report-details} +[状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/fast_test.html) + +#### 状态页文件 {#status-page-files} +- `runlog.out.log` 是包含所有其他日志的通用日志. +- `test_log.txt` +- `submodule_log.txt` 包含关于克隆和检查所需子模块的消息. +- `stderr.log` +- `stdout.log` +- `clickhouse-server.log` +- `clone_log.txt` +- `install_log.txt` +- `clickhouse-server.err.log` +- `build_log.txt` +- `cmake_log.txt` 包含关于C/C++和Linux标志检查的消息. + +#### 状态页列信息 {#status-page-columns} +- 测试名称 -- 包含测试的名称(不带路径, 例如, 所有类型的测试将被剥离到该名称). +- 测试状态 -- 跳过、成功或失败之一. +- 测试时间, 秒. -- 这个测试是空的. + +## 建构检查 {#build-check} +在各种配置中构建ClickHouse, 以便在后续步骤中使用. 您必须修复失败的构建.构建日志通常有足够的信息来修复错误, 但是您可能必须在本地重现故障. `cmake`选项可以在构建日志中通过grep `cmake`操作找到.使用这些选项并遵循[一般的构建过程](./build.md). + +### 报告详情 {#report-details} +[状态页示例](https://clickhouse-builds.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/clickhouse_build_check/report.html) +- **Compiler**: `gcc-9` 或 `clang-10` (或其他架构的`clang-10-xx`, 比如`clang-10-freebsd`). +- **Build type**: `Debug` or `RelWithDebInfo` (cmake). +- **Sanitizer**: `none` (without sanitizers), `address` (ASan), `memory` (MSan), `undefined` (UBSan), or `thread` (TSan). +- **Bundled**: `bundled` 构建使用来自 `contrib` 库, 而 `unbundled` 构建使用系统库. +- **Splitted**: `splitted` is a [split build](https://clickhouse.com/docs/en/development/build/#split-build) +- **Status**: `成功` 或 `失败` +- **Build log**: 链接到构建和文件复制日志, 当构建失败时很有用. +- **Build time**. +- **Artifacts**: 构建结果文件 (`XXX`是服务器版本, 比如`20.8.1.4344`). + - `clickhouse-client_XXX_all.deb` + -` clickhouse-common-static-dbg_XXX[+asan, +msan, +ubsan, +tsan]_amd64.deb` + - `clickhouse-common-staticXXX_amd64.deb` + - `clickhouse-server_XXX_all.deb` + - `clickhouse-test_XXX_all.deb` + - `clickhouse_XXX_amd64.buildinfo` + - `clickhouse_XXX_amd64.changes` + - `clickhouse`: Main built binary. + - `clickhouse-odbc-bridge` + - `unit_tests_dbms`: 带有 ClickHouse 单元测试的 GoogleTest 二进制文件. + - `shared_build.tgz`: 使用共享库构建. + - `performance.tgz`: 用于性能测试的特殊包. + +## 特殊构建检查 {#special-buildcheck} +使用clang-tidy执行静态分析和代码样式检查. 该报告类似于构建检查. 修复在构建日志中发现的错误. + +## 功能无状态测试 {#functional-stateless-tests} +为构建在不同配置中的ClickHouse二进制文件运行[无状态功能测试](./tests.md#functional-tests)——发布、调试、使用杀毒软件等.通过报告查看哪些测试失败,然后按照[此处](./tests.md#functional-test-locally)描述的在本地重现失败.注意, 您必须使用正确的构建配置来重现——在AddressSanitizer下测试可能失败,但在Debug中可以通过.从[CI构建检查页面](./build.md#you-dont-have-to-build-clickhouse)下载二进制文件, 或者在本地构建它. + +## 功能有状态测试 {#functional-stateful-tests} +运行[有状态功能测试](./tests.md#functional-tests).以无状态功能测试相同的方式对待它们.不同之处在于它们需要从[Yandex.Metrica数据集](https://clickhouse.com/docs/en/getting-started/example-datasets/metrica/)的`hits`和`visits`表来运行. + +## 集成测试 {#integration-tests} +运行[集成测试](./tests.md#integration-tests). + +## Testflows 检查{#testflows-check} +使用Testflows测试系统去运行一些测试, 在[此处](../../../tests/testflows/README.md#running-tests-locally)查看如何在本地运行它们. + +## 压力测试 {#stress-test} +从多个客户端并发运行无状态功能测试, 用以检测与并发相关的错误.如果失败: +``` +* Fix all other test failures first; +* Look at the report to find the server logs and check them for possible causes + of error. +``` + +## 冒烟测试 {#split-build-smoke-test} +检查[拆分构建](./build.md#split-build)配置中的服务器构建是否可以启动并运行简单查询.如果失败: +``` +* Fix other test errors first; +* Build the server in [split build](./build.md#split-build) configuration + locally and check whether it can start and run `select 1`. +``` + +## 兼容性检查 {#compatibility-check} +检查`clickhouse`二进制文件是否可以在带有旧libc版本的发行版上运行.如果失败, 请向维护人员寻求帮助. + +## AST模糊器 {#ast-fuzzer} +运行随机生成的查询来捕获程序错误.如果失败, 请向维护人员寻求帮助. + +## 性能测试 {#performance-tests} +测量查询性能的变化. 这是最长的检查, 只需不到 6 小时即可运行.性能测试报告在[此处](../../../docker/test/performance-comparison/README.md#how-to-read-the-report)有详细描述. + +## 质量保证 {#qa} +什么是状态页面上的任务(专用网络)项目? + +它是 Yandex 内部工作系统的链接. Yandex 员工可以看到检查的开始时间及其更详细的状态. + +运行测试的地方 + +Yandex 内部基础设施的某个地方. From 671ed9d4301211bf49e8629365dc0e0e33347248 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Tue, 26 Oct 2021 10:23:24 +0800 Subject: [PATCH 091/173] Adjusting invalid links --- docs/zh/development/continuous-integration.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/zh/development/continuous-integration.md b/docs/zh/development/continuous-integration.md index 41e22074915..6cff83067de 100644 --- a/docs/zh/development/continuous-integration.md +++ b/docs/zh/development/continuous-integration.md @@ -1,5 +1,5 @@ # 持续集成检查 {#continuous-integration-checks} -当你提交一个pull请求时, ClickHouse[持续集成(CI)系统](../tests.md#test-automation)会对您的代码运行一些自动检查. +当你提交一个pull请求时, ClickHouse[持续集成(CI)系统](https://clickhouse.com/docs/en/development/tests/#test-automation)会对您的代码运行一些自动检查. 这在存储库维护者(来自ClickHouse团队的人)筛选了您的代码并将可测试标签添加到您的pull请求之后发生. @@ -28,7 +28,7 @@ git push - `docs_output.txt`包含构建日志信息. [成功结果案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check/docs_output.txt) ## 描述信息检查 {#description-check} -检查pull请求的描述是否符合[PULL_REQUEST_TEMPLATE.md](../../../.github/PULL_REQUEST_TEMPLATE.md)模板. +检查pull请求的描述是否符合[PULL_REQUEST_TEMPLATE.md](https://github.com/ClickHouse/ClickHouse/blob/master/.github/PULL_REQUEST_TEMPLATE.md)模板. 您必须为您的更改指定一个更改日志类别(例如,Bug修复), 并且为[CHANGELOG.md](../whats-new/changelog/)编写一条用户可读的消息用来描述更改. @@ -118,7 +118,7 @@ git push 运行[集成测试](./tests.md#integration-tests). ## Testflows 检查{#testflows-check} -使用Testflows测试系统去运行一些测试, 在[此处](../../../tests/testflows/README.md#running-tests-locally)查看如何在本地运行它们. +使用Testflows测试系统去运行一些测试, 在[此处](https://github.com/ClickHouse/ClickHouse/tree/master/tests/testflows#running-tests-locally)查看如何在本地运行它们. ## 压力测试 {#stress-test} 从多个客户端并发运行无状态功能测试, 用以检测与并发相关的错误.如果失败: @@ -143,7 +143,7 @@ git push 运行随机生成的查询来捕获程序错误.如果失败, 请向维护人员寻求帮助. ## 性能测试 {#performance-tests} -测量查询性能的变化. 这是最长的检查, 只需不到 6 小时即可运行.性能测试报告在[此处](../../../docker/test/performance-comparison/README.md#how-to-read-the-report)有详细描述. +测量查询性能的变化. 这是最长的检查, 只需不到 6 小时即可运行.性能测试报告在[此处](https://github.com/ClickHouse/ClickHouse/tree/master/docker/test/performance-comparison#how-to-read-the-report)有详细描述. ## 质量保证 {#qa} 什么是状态页面上的任务(专用网络)项目? From 42c16f5697e79e0d9140202f5b891fe3a838aff7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Tue, 26 Oct 2021 10:24:38 +0800 Subject: [PATCH 092/173] Adjusting invalid links --- docs/zh/operations/external-authenticators/kerberos.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/external-authenticators/kerberos.md b/docs/zh/operations/external-authenticators/kerberos.md index 1121a86a793..5d2122c5fea 100644 --- a/docs/zh/operations/external-authenticators/kerberos.md +++ b/docs/zh/operations/external-authenticators/kerberos.md @@ -93,7 +93,7 @@ Kerberos主体名称格式通常遵循以下模式: 请注意, 现在, 一旦用户 `my_user` 使用 `kerberos`, 必须在主 `config.xml` 文件中启用 Kerberos,如前所述. ### 使用 SQL 启用 Kerberos {#enabling-kerberos-using-sql} -在 ClickHouse 中启用 [SQL 驱动的访问控制和帐户管理](./access-rights.md#access-control)后, 也可以使用 SQL 语句创建由 Kerberos 识别的用户. +在 ClickHouse 中启用 [SQL 驱动的访问控制和帐户管理](https://clickhouse.com/docs/en/operations/access-rights/#access-control)后, 也可以使用 SQL 语句创建由 Kerberos 识别的用户. ```sql CREATE USER my_user IDENTIFIED WITH kerberos REALM 'EXAMPLE.COM' From 97bcceca445cbc1e07e1d2aa9e232610fadac7eb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Oct 2021 09:49:12 +0300 Subject: [PATCH 093/173] Improve usability of remote_url_allow_hosts --- src/Common/RemoteHostFilter.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/RemoteHostFilter.cpp b/src/Common/RemoteHostFilter.cpp index 73c84364f3c..ba7f163fd16 100644 --- a/src/Common/RemoteHostFilter.cpp +++ b/src/Common/RemoteHostFilter.cpp @@ -18,14 +18,14 @@ void RemoteHostFilter::checkURL(const Poco::URI & uri) const { if (!checkForDirectEntry(uri.getHost()) && !checkForDirectEntry(uri.getHost() + ":" + toString(uri.getPort()))) - throw Exception("URL \"" + uri.toString() + "\" is not allowed in config.xml", ErrorCodes::UNACCEPTABLE_URL); + throw Exception("URL \"" + uri.toString() + "\" is not allowed in configuration file, see ", ErrorCodes::UNACCEPTABLE_URL); } void RemoteHostFilter::checkHostAndPort(const std::string & host, const std::string & port) const { if (!checkForDirectEntry(host) && !checkForDirectEntry(host + ":" + port)) - throw Exception("URL \"" + host + ":" + port + "\" is not allowed in config.xml", ErrorCodes::UNACCEPTABLE_URL); + throw Exception("URL \"" + host + ":" + port + "\" is not allowed in configuration file, see ", ErrorCodes::UNACCEPTABLE_URL); } void RemoteHostFilter::setValuesFromConfig(const Poco::Util::AbstractConfiguration & config) From da3e13b730951b87a1ca9087da09ad9fc76e7fa3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Oct 2021 10:00:19 +0300 Subject: [PATCH 094/173] Fix indecency #30337 --- src/Common/ProgressIndication.cpp | 24 +++++++++++++----------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index db1daaac415..26dae8d5b73 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -14,17 +14,19 @@ namespace { - constexpr UInt64 ZERO = 0; + constexpr UInt64 ALL_THREADS = 0; UInt64 calculateNewCoresNumber(DB::ThreadIdToTimeMap const & prev, DB::ThreadIdToTimeMap const& next) { - if (next.find(ZERO) == next.end()) - return ZERO; - auto accumulated = std::accumulate(next.cbegin(), next.cend(), ZERO, - [&prev](UInt64 acc, auto const & elem) + if (next.find(ALL_THREADS) == next.end()) + return 0; + + auto accumulated = std::accumulate(next.cbegin(), next.cend(), 0, + [&prev](UInt64 acc, const auto & elem) { - if (elem.first == ZERO) + if (elem.first == ALL_THREADS) return acc; + auto thread_time = elem.second.time(); auto it = prev.find(elem.first); if (it != prev.end()) @@ -32,9 +34,9 @@ namespace return acc + thread_time; }); - auto elapsed = next.at(ZERO).time() - (prev.contains(ZERO) ? prev.at(ZERO).time() : ZERO); - if (elapsed == ZERO) - return ZERO; + auto elapsed = next.at(ALL_THREADS).time() - (prev.contains(ALL_THREADS) ? prev.at(ALL_THREADS).time() : 0); + if (elapsed == 0) + return 0; return (accumulated + elapsed - 1) / elapsed; } } @@ -109,7 +111,7 @@ size_t ProgressIndication::getUsedThreadsCount() const UInt64 ProgressIndication::getApproximateCoresNumber() const { - return std::accumulate(host_active_cores.cbegin(), host_active_cores.cend(), ZERO, + return std::accumulate(host_active_cores.cbegin(), host_active_cores.cend(), 0, [](UInt64 acc, auto const & elem) { return acc + elem.second; @@ -125,7 +127,7 @@ ProgressIndication::MemoryUsage ProgressIndication::getMemoryUsage() const // In ProfileEvents packets thread id 0 specifies common profiling information // for all threads executing current query on specific host. So instead of summing per thread // memory consumption it's enough to look for data with thread id 0. - if (auto it = host_data.second.find(ZERO); it != host_data.second.end()) + if (auto it = host_data.second.find(ALL_THREADS); it != host_data.second.end()) host_usage = it->second.memory_usage; return MemoryUsage{.total = acc.total + host_usage, .max = std::max(acc.max, host_usage)}; }); From c8f9bab93bcb37b7d30b6dd83d540161f3e8cd5e Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 Oct 2021 11:28:28 +0300 Subject: [PATCH 095/173] Better wait for server start in integration tests --- tests/integration/helpers/cluster.py | 29 +++++++++++++++++++++++----- 1 file changed, 24 insertions(+), 5 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 3854cadaba5..c7b140b6471 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2056,12 +2056,31 @@ class ClickHouseInstance: def start_clickhouse(self, start_wait_sec=30): if not self.stay_alive: - raise Exception("clickhouse can be started again only with stay_alive=True instance") + raise Exception("ClickHouse can be started again only with stay_alive=True instance") + + time_to_sleep = 0.5 + start_tries = 5 + + total_tries = int(start_wait_sec / time_to_sleep) + query_tries = int(total_tries / start_tries) + + for i in range(start_tries): + # sometimes after SIGKILL (hard reset) server may refuse to start for some time + # for different reasons + self.exec_in_container(["bash", "-c", "{} --daemon".format(self.clickhouse_start_command)], user=str(os.getuid())) + started = False + for _ in range(query_tries): + try: + self.query("select 1") + started = True + break + except: + time.sleep(time_to_sleep) + if started: + break + else: + raise Exception("Cannot start ClickHouse, se additional info in logs") - self.exec_in_container(["bash", "-c", "{} --daemon".format(self.clickhouse_start_command)], user=str(os.getuid())) - # wait start - from helpers.test_tools import assert_eq_with_retry - assert_eq_with_retry(self, "select 1", "1", retry_count=int(start_wait_sec / 0.5), sleep_time=0.5) def restart_clickhouse(self, stop_start_wait_sec=30, kill=False): self.stop_clickhouse(stop_start_wait_sec, kill) From b7ed729c81f904b70a7d71c8498d3bead4b823e3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 Oct 2021 11:29:22 +0300 Subject: [PATCH 096/173] typo --- tests/integration/helpers/cluster.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c7b140b6471..9d7657d3a83 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2066,7 +2066,7 @@ class ClickHouseInstance: for i in range(start_tries): # sometimes after SIGKILL (hard reset) server may refuse to start for some time - # for different reasons + # for different reasons. self.exec_in_container(["bash", "-c", "{} --daemon".format(self.clickhouse_start_command)], user=str(os.getuid())) started = False for _ in range(query_tries): @@ -2079,7 +2079,7 @@ class ClickHouseInstance: if started: break else: - raise Exception("Cannot start ClickHouse, se additional info in logs") + raise Exception("Cannot start ClickHouse, see additional info in logs") def restart_clickhouse(self, stop_start_wait_sec=30, kill=False): From 3b5b1d20061304c65049cdb67c5b9e473d224440 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 Oct 2021 12:07:07 +0300 Subject: [PATCH 097/173] Fix keeper zookeeper converter test --- .../test_keeper_zookeeper_converter/configs/keeper_config.xml | 4 ++-- tests/integration/test_keeper_zookeeper_converter/test.py | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_keeper_zookeeper_converter/configs/keeper_config.xml b/tests/integration/test_keeper_zookeeper_converter/configs/keeper_config.xml index d98c8d72e61..612cb979bb7 100644 --- a/tests/integration/test_keeper_zookeeper_converter/configs/keeper_config.xml +++ b/tests/integration/test_keeper_zookeeper_converter/configs/keeper_config.xml @@ -6,8 +6,8 @@ /var/lib/clickhouse/coordination/snapshots - 5000 - 10000 + 30000 + 600000 trace 75 diff --git a/tests/integration/test_keeper_zookeeper_converter/test.py b/tests/integration/test_keeper_zookeeper_converter/test.py index eac2b4c45c5..6829b4a9000 100644 --- a/tests/integration/test_keeper_zookeeper_converter/test.py +++ b/tests/integration/test_keeper_zookeeper_converter/test.py @@ -60,12 +60,12 @@ def started_cluster(): finally: cluster.shutdown() -def get_fake_zk(timeout=30.0): +def get_fake_zk(timeout=60.0): _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip('node') + ":9181", timeout=timeout) _fake_zk_instance.start() return _fake_zk_instance -def get_genuine_zk(timeout=30.0): +def get_genuine_zk(timeout=60.0): _genuine_zk_instance = KazooClient(hosts=cluster.get_instance_ip('node') + ":2181", timeout=timeout) _genuine_zk_instance.start() return _genuine_zk_instance From 73bbf7549af0e5f11c8f0b858f466fadc9b15502 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 21 Oct 2021 17:13:39 +0300 Subject: [PATCH 098/173] HashedArrayDictionary optimize read multiple attributes --- src/Dictionaries/HashedArrayDictionary.cpp | 210 +++++++++++++++++++++ src/Dictionaries/HashedArrayDictionary.h | 22 +++ 2 files changed, 232 insertions(+) diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index 426857e055f..0ba73b8bd72 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -146,6 +146,85 @@ ColumnPtr HashedArrayDictionary::getColumn( return result; } +template +Columns HashedArrayDictionary::getColumns( + const Strings & attribute_names, + const DataTypes & result_types, + const Columns & key_columns, + const DataTypes & key_types, + const Columns & default_values_columns) const +{ + if (dictionary_key_type == DictionaryKeyType::Complex) + dict_struct.validateKeyTypes(key_types); + + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor extractor(key_columns, arena_holder.getComplexKeyArena()); + + const size_t keys_size = extractor.getKeysSize(); + + PaddedPODArray key_index_to_element_index; + + /** Optimization for multiple attributes. + * For each key save element index in key_index_to_element_index array. + * Later in type_call for attribute use getItemsImpl specialization with key_index_to_element_index array + * intead of DictionaryKeyExtractor. + */ + if (attribute_names.size() > 1) { + const auto & key_attribute_container = key_attribute.container; + size_t keys_found = 0; + + key_index_to_element_index.resize(keys_size); + + for (size_t key_index = 0; key_index < keys_size; ++key_index) + { + auto key = extractor.extractCurrentKey(); + + auto it = key_attribute_container.find(key); + if (it == key_attribute_container.end()) + { + key_index_to_element_index[key_index] = -1; + } + else + { + key_index_to_element_index[key_index] = it->getMapped(); + ++keys_found; + } + + extractor.rollbackCurrentKey(); + } + + query_count.fetch_add(keys_size, std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); + } + + size_t attribute_names_size = attribute_names.size(); + + Columns result_columns; + result_columns.reserve(attribute_names_size); + + for (size_t i = 0; i < attribute_names_size; ++i) + { + ColumnPtr result_column; + + const auto & attribute_name = attribute_names[i]; + const auto & result_type = result_types[i]; + const auto & default_values_column = default_values_columns[i]; + + const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); + const size_t attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second; + auto & attribute = attributes[attribute_index]; + + if (attribute_names_size > 1) + result_column = getAttributeColumn(attribute, dictionary_attribute, keys_size, default_values_column, key_index_to_element_index); + else + result_column = getAttributeColumn(attribute, dictionary_attribute, keys_size, default_values_column, extractor); + + result_columns.emplace_back(std::move(result_column)); + } + + return result_columns; +} + template ColumnUInt8::Ptr HashedArrayDictionary::hasKeys(const Columns & key_columns, const DataTypes & key_types) const { @@ -499,6 +578,102 @@ void HashedArrayDictionary::resize(size_t added_rows) key_attribute.container.reserve(added_rows); } +template +template +ColumnPtr HashedArrayDictionary::getAttributeColumn( + const Attribute & attribute, + const DictionaryAttribute & dictionary_attribute, + size_t keys_size, + ColumnPtr default_values_column, + KeysProvider && keys_object) const +{ + ColumnPtr result; + + bool is_attribute_nullable = attribute.is_index_null.has_value(); + + ColumnUInt8::MutablePtr col_null_map_to; + ColumnUInt8::Container * vec_null_map_to = nullptr; + if (attribute.is_index_null) + { + col_null_map_to = ColumnUInt8::create(keys_size, false); + vec_null_map_to = &col_null_map_to->getData(); + } + + auto type_call = [&](const auto & dictionary_attribute_type) + { + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; + using ColumnProvider = DictionaryAttributeColumnProvider; + + DictionaryDefaultValueExtractor default_value_extractor(dictionary_attribute.null_value, default_values_column); + + auto column = ColumnProvider::getColumn(dictionary_attribute, keys_size); + + if constexpr (std::is_same_v) + { + auto * out = column.get(); + + getItemsImpl( + attribute, + keys_object, + [&](const size_t, const Array & value, bool) { out->insert(value); }, + default_value_extractor); + } + else if constexpr (std::is_same_v) + { + auto * out = column.get(); + + if (is_attribute_nullable) + getItemsImpl( + attribute, + keys_object, + [&](size_t row, const StringRef value, bool is_null) + { + (*vec_null_map_to)[row] = is_null; + out->insertData(value.data, value.size); + }, + default_value_extractor); + else + getItemsImpl( + attribute, + keys_object, + [&](size_t, const StringRef value, bool) { out->insertData(value.data, value.size); }, + default_value_extractor); + } + else + { + auto & out = column->getData(); + + if (is_attribute_nullable) + getItemsImpl( + attribute, + keys_object, + [&](size_t row, const auto value, bool is_null) + { + (*vec_null_map_to)[row] = is_null; + out[row] = value; + }, + default_value_extractor); + else + getItemsImpl( + attribute, + keys_object, + [&](size_t row, const auto value, bool) { out[row] = value; }, + default_value_extractor); + } + + result = std::move(column); + }; + + callOnDictionaryAttributeType(attribute.type, type_call); + + if (is_attribute_nullable) + result = ColumnNullable::create(std::move(result), std::move(col_null_map_to)); + + return result; +} + template template void HashedArrayDictionary::getItemsImpl( @@ -547,6 +722,41 @@ void HashedArrayDictionary::getItemsImpl( found_count.fetch_add(keys_found, std::memory_order_relaxed); } +template +template +void HashedArrayDictionary::getItemsImpl( + const Attribute & attribute, + const PaddedPODArray & key_index_to_element_index, + ValueSetter && set_value, + DefaultValueExtractor & default_value_extractor) const +{ + const auto & attribute_container = std::get>(attribute.container); + const size_t keys_size = key_index_to_element_index.size(); + + for (size_t key_index = 0; key_index < keys_size; ++key_index) + { + bool key_exists = key_index_to_element_index[key_index] != -1; + + if (key_exists) + { + size_t element_index = static_cast(key_index_to_element_index[key_index]); + const auto & element = attribute_container[element_index]; + + if constexpr (is_nullable) + set_value(key_index, element, (*attribute.is_index_null)[element_index]); + else + set_value(key_index, element, false); + } + else + { + if constexpr (is_nullable) + set_value(key_index, default_value_extractor[key_index], default_value_extractor.isNullAt(key_index)); + else + set_value(key_index, default_value_extractor[key_index], false); + } + } +} + template StringRef HashedArrayDictionary::copyKeyInArena(StringRef key) { diff --git a/src/Dictionaries/HashedArrayDictionary.h b/src/Dictionaries/HashedArrayDictionary.h index 0af3fb65581..5ad1efeb056 100644 --- a/src/Dictionaries/HashedArrayDictionary.h +++ b/src/Dictionaries/HashedArrayDictionary.h @@ -93,6 +93,13 @@ public: const DataTypes & key_types, const ColumnPtr & default_values_column) const override; + Columns getColumns( + const Strings & attribute_names, + const DataTypes & result_types, + const Columns & key_columns, + const DataTypes & key_types, + const Columns & default_values_columns) const override; + ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::Simple && dict_struct.hierarchical_attribute_index.has_value(); } @@ -170,6 +177,14 @@ private: void calculateBytesAllocated(); + template + ColumnPtr getAttributeColumn( + const Attribute & attribute, + const DictionaryAttribute & dictionary_attribute, + size_t keys_size, + ColumnPtr default_values_column, + KeysProvider && keys_object) const; + template void getItemsImpl( const Attribute & attribute, @@ -177,6 +192,13 @@ private: ValueSetter && set_value, DefaultValueExtractor & default_value_extractor) const; + template + void getItemsImpl( + const Attribute & attribute, + const PaddedPODArray & key_index_to_element_index, + ValueSetter && set_value, + DefaultValueExtractor & default_value_extractor) const; + template void getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func); From 6cf16862abae23bc25366b78b6f841d3abec3a44 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 22 Oct 2021 11:03:40 +0300 Subject: [PATCH 099/173] Fixed style check --- src/Dictionaries/HashedArrayDictionary.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index 0ba73b8bd72..94145e8fa99 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -167,9 +167,10 @@ Columns HashedArrayDictionary::getColumns( /** Optimization for multiple attributes. * For each key save element index in key_index_to_element_index array. * Later in type_call for attribute use getItemsImpl specialization with key_index_to_element_index array - * intead of DictionaryKeyExtractor. + * instead of DictionaryKeyExtractor. */ - if (attribute_names.size() > 1) { + if (attribute_names.size() > 1) + { const auto & key_attribute_container = key_attribute.container; size_t keys_found = 0; From dbf481102986780667fac1da04b91ae8258450cd Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 26 Oct 2021 13:00:42 +0300 Subject: [PATCH 100/173] Updated documentation --- src/Common/LRUCache.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 3df27cbdce9..bbc09fd3aff 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -36,6 +36,9 @@ public: using Mapped = TMapped; using MappedPtr = std::shared_ptr; + /** Initialize LRUCache with max_size and max_elements_size. + * max_elements_size == 0 means no elements size restrictions. + */ LRUCache(size_t max_size_, size_t max_elements_size_ = 0) : max_size(std::max(static_cast(1), max_size_)) , max_elements_size(max_elements_size_) From a1cab43feb52936e6969fa7d119733efe7808407 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 26 Oct 2021 13:32:07 +0300 Subject: [PATCH 101/173] fix five years old bug in StingSearcher --- src/Common/StringSearcher.h | 133 ++++++------------ ...163_search_case_insensetive_utf8.reference | 12 ++ .../01163_search_case_insensetive_utf8.sql | 12 ++ 3 files changed, 66 insertions(+), 91 deletions(-) create mode 100644 tests/queries/0_stateless/01163_search_case_insensetive_utf8.reference create mode 100644 tests/queries/0_stateless/01163_search_case_insensetive_utf8.sql diff --git a/src/Common/StringSearcher.h b/src/Common/StringSearcher.h index af1d36adf81..f34bc6f7322 100644 --- a/src/Common/StringSearcher.h +++ b/src/Common/StringSearcher.h @@ -116,9 +116,9 @@ public: /// lower and uppercase variants of the first octet of the first character in `needle` size_t length_l = UTF8::convertCodePointToUTF8(first_l_u32, l_seq, sizeof(l_seq)); - size_t length_r = UTF8::convertCodePointToUTF8(first_u_u32, u_seq, sizeof(u_seq)); + size_t length_u = UTF8::convertCodePointToUTF8(first_u_u32, u_seq, sizeof(u_seq)); - if (length_l != length_r) + if (length_l != length_u) throw Exception{"UTF8 sequences with different lowercase and uppercase lengths are not supported", ErrorCodes::UNSUPPORTED_PARAMETER}; } @@ -183,6 +183,31 @@ public: #endif } + template > + ALWAYS_INLINE bool compareTrivial(const CharT * haystack_pos, const CharT * const haystack_end, const uint8_t * needle_pos) const + { + while (haystack_pos < haystack_end && needle_pos < needle_end) + { + auto haystack_code_point = UTF8::convertUTF8ToCodePoint(haystack_pos, haystack_end - haystack_pos); + auto needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos); + + /// Invalid UTF-8, should not compare equals + if (!haystack_code_point || !needle_code_point) + break; + + /// Not equals case insensitive. + if (Poco::Unicode::toLower(*haystack_code_point) != Poco::Unicode::toLower(*needle_code_point)) + break; + + /// @note assuming sequences for lowercase and uppercase have exact same length (that is not always true) + const auto len = UTF8::seqLength(*haystack_pos); + haystack_pos += len; + needle_pos += len; + } + + return needle_pos == needle_end; + } + template > ALWAYS_INLINE bool compare(const CharT * /*haystack*/, const CharT * haystack_end, const CharT * pos) const { @@ -200,34 +225,15 @@ public: { if (mask == cachemask) { - pos += cache_valid_len; - auto needle_pos = needle + cache_valid_len; - - while (needle_pos < needle_end) - { - auto haystack_code_point = UTF8::convertUTF8ToCodePoint(pos, haystack_end - pos); - auto needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos); - - /// Invalid UTF-8, should not compare equals - if (!haystack_code_point || !needle_code_point) - break; - - /// Not equals case insensitive. - if (Poco::Unicode::toLower(*haystack_code_point) != Poco::Unicode::toLower(*needle_code_point)) - break; - - /// @note assuming sequences for lowercase and uppercase have exact same length (that is not always true) - const auto len = UTF8::seqLength(*pos); - pos += len; - needle_pos += len; - } - - if (needle_pos == needle_end) + if (compareTrivial(pos, haystack_end, needle)) return true; } } else if ((mask & cachemask) == cachemask) - return true; + { + if (compareTrivial(pos, haystack_end, needle)) + return true; + } return false; } @@ -238,25 +244,7 @@ public: pos += first_needle_symbol_is_ascii; auto needle_pos = needle + first_needle_symbol_is_ascii; - while (needle_pos < needle_end) - { - auto haystack_code_point = UTF8::convertUTF8ToCodePoint(pos, haystack_end - pos); - auto needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos); - - /// Invalid UTF-8, should not compare equals - if (!haystack_code_point || !needle_code_point) - break; - - /// Not equals case insensitive. - if (Poco::Unicode::toLower(*haystack_code_point) != Poco::Unicode::toLower(*needle_code_point)) - break; - - const auto len = UTF8::seqLength(*pos); - pos += len; - needle_pos += len; - } - - if (needle_pos == needle_end) + if (compareTrivial(pos, haystack_end, needle_pos)) return true; } @@ -299,40 +287,21 @@ public: const auto v_against_l_offset = _mm_cmpeq_epi8(v_haystack_offset, cachel); const auto v_against_u_offset = _mm_cmpeq_epi8(v_haystack_offset, cacheu); const auto v_against_l_or_u_offset = _mm_or_si128(v_against_l_offset, v_against_u_offset); - const auto mask_offset = _mm_movemask_epi8(v_against_l_or_u_offset); + const auto mask_offset_both = _mm_movemask_epi8(v_against_l_or_u_offset); if (0xffff == cachemask) { - if (mask_offset == cachemask) + if (mask_offset_both == cachemask) { - auto haystack_pos = haystack + cache_valid_len; - auto needle_pos = needle + cache_valid_len; - - while (haystack_pos < haystack_end && needle_pos < needle_end) - { - auto haystack_code_point = UTF8::convertUTF8ToCodePoint(haystack_pos, haystack_end - haystack_pos); - auto needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos); - - /// Invalid UTF-8, should not compare equals - if (!haystack_code_point || !needle_code_point) - break; - - /// Not equals case insensitive. - if (Poco::Unicode::toLower(*haystack_code_point) != Poco::Unicode::toLower(*needle_code_point)) - break; - - /// @note assuming sequences for lowercase and uppercase have exact same length (that is not always true) - const auto len = UTF8::seqLength(*haystack_pos); - haystack_pos += len; - needle_pos += len; - } - - if (needle_pos == needle_end) + if (compareTrivial(haystack, haystack_end, needle)) return haystack; } } - else if ((mask_offset & cachemask) == cachemask) - return haystack; + else if ((mask_offset_both & cachemask) == cachemask) + { + if (compareTrivial(haystack, haystack_end, needle)) + return haystack; + } /// first octet was ok, but not the first 16, move to start of next sequence and reapply haystack += UTF8::seqLength(*haystack); @@ -349,25 +318,7 @@ public: auto haystack_pos = haystack + first_needle_symbol_is_ascii; auto needle_pos = needle + first_needle_symbol_is_ascii; - while (haystack_pos < haystack_end && needle_pos < needle_end) - { - auto haystack_code_point = UTF8::convertUTF8ToCodePoint(haystack_pos, haystack_end - haystack_pos); - auto needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos); - - /// Invalid UTF-8, should not compare equals - if (!haystack_code_point || !needle_code_point) - break; - - /// Not equals case insensitive. - if (Poco::Unicode::toLower(*haystack_code_point) != Poco::Unicode::toLower(*needle_code_point)) - break; - - const auto len = UTF8::seqLength(*haystack_pos); - haystack_pos += len; - needle_pos += len; - } - - if (needle_pos == needle_end) + if (compareTrivial(haystack_pos, haystack_end, needle_pos)) return haystack; } diff --git a/tests/queries/0_stateless/01163_search_case_insensetive_utf8.reference b/tests/queries/0_stateless/01163_search_case_insensetive_utf8.reference new file mode 100644 index 00000000000..66f4ca4a5a8 --- /dev/null +++ b/tests/queries/0_stateless/01163_search_case_insensetive_utf8.reference @@ -0,0 +1,12 @@ +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/01163_search_case_insensetive_utf8.sql b/tests/queries/0_stateless/01163_search_case_insensetive_utf8.sql new file mode 100644 index 00000000000..99bdd38ceae --- /dev/null +++ b/tests/queries/0_stateless/01163_search_case_insensetive_utf8.sql @@ -0,0 +1,12 @@ +SELECT positionCaseInsensitiveUTF8(materialize('сссссс'), 'Ё'); +SELECT countSubstringsCaseInsensitiveUTF8(materialize('сссссс'), 'ё'); +SELECT positionCaseInsensitiveUTF8(materialize('сссссссс'), 'ё'); +SELECT countSubstringsCaseInsensitiveUTF8(materialize('сссссссс'), 'Ё'); +SELECT countSubstringsCaseInsensitiveUTF8(materialize('ссссссссссссссссссс'), 'ёёёёёёё'); +SELECT positionCaseInsensitiveUTF8(materialize('ссссссссссссссссссс'), 'ёЁёЁёЁё'); +SELECT countSubstringsCaseInsensitiveUTF8(materialize('ссссссссссссссссссс'), 'ёЁёЁёЁёЁёЁ'); +SELECT positionCaseInsensitiveUTF8(materialize('ссссссссссссссссссс'), 'ЁЁЁЁЁЁЁЁЁЁ'); +SELECT countSubstringsCaseInsensitiveUTF8(materialize('ссссссссссссссссссс'), 'ёЁёЁёЁёссс'); +SELECT positionCaseInsensitiveUTF8(materialize('ссссссссссссссссссс'), 'ёЁёЁёЁёссс'); +SELECT countSubstringsCaseInsensitiveUTF8(materialize('ссссссссссссссссссс'), 'ЁС'); +SELECT positionCaseInsensitiveUTF8(materialize('ссссссссссссссссссс'), 'ёс'); From 2e7e195e776dbfeb3c5832852ee277187e4e50db Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 25 Oct 2021 20:49:49 +0300 Subject: [PATCH 102/173] change alter_lock to std::timed_mutex --- src/Interpreters/InterpreterAlterQuery.cpp | 3 ++- src/Storages/IStorage.cpp | 18 +++++++++++------- src/Storages/IStorage.h | 14 ++++++-------- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageBuffer.h | 2 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageDistributed.h | 2 +- src/Storages/StorageMaterializedView.cpp | 2 +- src/Storages/StorageMaterializedView.h | 2 +- src/Storages/StorageMerge.cpp | 2 +- src/Storages/StorageMerge.h | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageNull.cpp | 2 +- src/Storages/StorageNull.h | 2 +- src/Storages/StorageProxy.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 7 ++++--- src/Storages/StorageReplicatedMergeTree.h | 2 +- src/Storages/TableLockHolder.h | 2 -- 21 files changed, 39 insertions(+), 37 deletions(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index ace82f47f05..8e84373cb0f 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -76,7 +76,8 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); if (table->isStaticStorage()) throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only"); - auto alter_lock = table->lockForAlter(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); + auto table_lock = table->lockForShare(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); + auto alter_lock = table->lockForAlter(getContext()->getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = table->getInMemoryMetadataPtr(); /// Add default database to table identifiers that we can encounter in e.g. default expressions, mutation expression, etc. diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 7c158794caf..dd851f19906 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -60,27 +60,31 @@ TableLockHolder IStorage::lockForShare(const String & query_id, const std::chron return result; } -TableLockHolder IStorage::lockForAlter(const String & query_id, const std::chrono::milliseconds & acquire_timeout) +IStorage::AlterLockHolder IStorage::lockForAlter(const std::chrono::milliseconds & acquire_timeout) { - TableLockHolder result = tryLockTimed(alter_lock, RWLockImpl::Write, query_id, acquire_timeout); + AlterLockHolder lock{alter_lock, std::defer_lock}; + + if (!lock.try_lock_for(acquire_timeout)) + throw Exception(ErrorCodes::DEADLOCK_AVOIDED, + "Locking attempt for ALTER on \"{}\" has timed out! ({} ms) " + "Possible deadlock avoided. Client should retry.", + getStorageID().getFullTableName(), std::to_string(acquire_timeout.count())); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); - return result; + return lock; } TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, const std::chrono::milliseconds & acquire_timeout) { TableExclusiveLockHolder result; - result.alter_lock = tryLockTimed(alter_lock, RWLockImpl::Write, query_id, acquire_timeout); + result.drop_lock = tryLockTimed(drop_lock, RWLockImpl::Write, query_id, acquire_timeout); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); - result.drop_lock = tryLockTimed(drop_lock, RWLockImpl::Write, query_id, acquire_timeout); - return result; } @@ -126,7 +130,7 @@ Pipe IStorage::alterPartition( throw Exception("Partition operations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } -void IStorage::alter(const AlterCommands & params, ContextPtr context, TableLockHolder &) +void IStorage::alter(const AlterCommands & params, ContextPtr context, AlterLockHolder &) { auto table_id = getStorageID(); StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 701cf127521..aaaf919d3bf 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -238,7 +238,8 @@ public: /// Lock table for alter. This lock must be acuqired in ALTER queries to be /// sure, that we execute only one simultaneous alter. Doesn't affect share lock. - TableLockHolder lockForAlter(const String & query_id, const std::chrono::milliseconds & acquire_timeout); + using AlterLockHolder = std::unique_lock; + AlterLockHolder lockForAlter(const std::chrono::milliseconds & acquire_timeout); /// Lock table exclusively. This lock must be acquired if you want to be /// sure, that no other thread (SELECT, merge, ALTER, etc.) doing something @@ -417,7 +418,7 @@ public: /** ALTER tables in the form of column changes that do not affect the change * to Storage or its parameters. Executes under alter lock (lockForAlter). */ - virtual void alter(const AlterCommands & params, ContextPtr context, TableLockHolder & alter_lock_holder); + virtual void alter(const AlterCommands & params, ContextPtr context, AlterLockHolder & alter_lock_holder); /** Checks that alter commands can be applied to storage. For example, columns can be modified, * or primary key can be changes, etc. @@ -590,12 +591,9 @@ public: virtual bool dropTableImmediately() { return false; } private: - /// Lock required for alter queries (lockForAlter). Always taken for write - /// (actually can be replaced with std::mutex, but for consistency we use - /// RWLock). Allows to execute only one simultaneous alter query. Also it - /// should be taken by DROP-like queries, to be sure, that all alters are - /// finished. - mutable RWLock alter_lock = RWLockImpl::create(); + /// Lock required for alter queries (lockForAlter). + /// Allows to execute only one simultaneous alter query. + mutable std::timed_mutex alter_lock; /// Lock required for drop queries. Every thread that want to ensure, that /// table is not dropped have to table this lock for read (lockForShare). diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 57729ef2eda..235ad79064a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2211,7 +2211,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createPart( void MergeTreeData::changeSettings( const ASTPtr & new_settings, - TableLockHolder & /* table_lock_holder */) + AlterLockHolder & /* table_lock_holder */) { if (new_settings) { diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 2ea6a89002c..22ec7ce6f53 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -583,7 +583,7 @@ public: /// Change MergeTreeSettings void changeSettings( const ASTPtr & new_settings, - TableLockHolder & table_lock_holder); + AlterLockHolder & table_lock_holder); /// Should be called if part data is suspected to be corrupted. void reportBrokenPart(const String & name) const diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index a74223054e9..7d886551a0e 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -1058,7 +1058,7 @@ std::optional StorageBuffer::totalBytes(const Settings & /*settings*/) c return total_writes.bytes; } -void StorageBuffer::alter(const AlterCommands & params, ContextPtr local_context, TableLockHolder &) +void StorageBuffer::alter(const AlterCommands & params, ContextPtr local_context, AlterLockHolder &) { auto table_id = getStorageID(); checkAlterIsPossible(params, local_context); diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 3e8955ad864..28f8de5fd88 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -107,7 +107,7 @@ public: void checkAlterIsPossible(const AlterCommands & commands, ContextPtr context) const override; /// The structure of the subordinate table is not checked and does not change. - void alter(const AlterCommands & params, ContextPtr context, TableLockHolder & table_lock_holder) override; + void alter(const AlterCommands & params, ContextPtr context, AlterLockHolder & table_lock_holder) override; std::optional totalRows(const Settings & settings) const override; std::optional totalBytes(const Settings & settings) const override; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 1dd75f09b92..2c528a4ebf5 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -785,7 +785,7 @@ void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, Co } } -void StorageDistributed::alter(const AlterCommands & params, ContextPtr local_context, TableLockHolder &) +void StorageDistributed::alter(const AlterCommands & params, ContextPtr local_context, AlterLockHolder &) { auto table_id = getStorageID(); diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 50e9c935a17..540ae413f19 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -94,7 +94,7 @@ public: /// in the sub-tables, you need to manually add and delete columns /// the structure of the sub-table is not checked - void alter(const AlterCommands & params, ContextPtr context, TableLockHolder & table_lock_holder) override; + void alter(const AlterCommands & params, ContextPtr context, AlterLockHolder & table_lock_holder) override; void startup() override; void shutdown() override; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 29dbd6d38ed..361ff95984f 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -274,7 +274,7 @@ bool StorageMaterializedView::optimize( void StorageMaterializedView::alter( const AlterCommands & params, ContextPtr local_context, - TableLockHolder &) + AlterLockHolder &) { auto table_id = getStorageID(); StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index b6f677e84bb..105f3f25533 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -50,7 +50,7 @@ public: const Names & deduplicate_by_columns, ContextPtr context) override; - void alter(const AlterCommands & params, ContextPtr context, TableLockHolder & table_lock_holder) override; + void alter(const AlterCommands & params, ContextPtr context, AlterLockHolder & table_lock_holder) override; void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 38db0b61e8d..979e027d338 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -651,7 +651,7 @@ void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, ContextP } void StorageMerge::alter( - const AlterCommands & params, ContextPtr local_context, TableLockHolder &) + const AlterCommands & params, ContextPtr local_context, AlterLockHolder &) { auto table_id = getStorageID(); diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index b9a127515f8..52a91747433 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -43,7 +43,7 @@ public: /// you need to add and remove columns in the sub-tables manually /// the structure of sub-tables is not checked - void alter(const AlterCommands & params, ContextPtr context, TableLockHolder & table_lock_holder) override; + void alter(const AlterCommands & params, ContextPtr context, AlterLockHolder & table_lock_holder) override; bool mayBenefitFromIndexForIn( const ASTPtr & left_in_operand, ContextPtr query_context, const StorageMetadataPtr & metadata_snapshot) const override; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 699c78cdab9..84022e3278d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -271,7 +271,7 @@ void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont void StorageMergeTree::alter( const AlterCommands & commands, ContextPtr local_context, - TableLockHolder & table_lock_holder) + AlterLockHolder & table_lock_holder) { auto table_id = getStorageID(); auto old_storage_settings = getSettings(); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index db651bd00c3..8ed4b707b34 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -86,7 +86,7 @@ public: void drop() override; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; - void alter(const AlterCommands & commands, ContextPtr context, TableLockHolder & table_lock_holder) override; + void alter(const AlterCommands & commands, ContextPtr context, AlterLockHolder & table_lock_holder) override; void checkTableCanBeDropped() const override; diff --git a/src/Storages/StorageNull.cpp b/src/Storages/StorageNull.cpp index 2b3585e360e..094e345e433 100644 --- a/src/Storages/StorageNull.cpp +++ b/src/Storages/StorageNull.cpp @@ -64,7 +64,7 @@ void StorageNull::checkAlterIsPossible(const AlterCommands & commands, ContextPt } -void StorageNull::alter(const AlterCommands & params, ContextPtr context, TableLockHolder &) +void StorageNull::alter(const AlterCommands & params, ContextPtr context, AlterLockHolder &) { auto table_id = getStorageID(); diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index de667c1d75c..5fef7f984e4 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -43,7 +43,7 @@ public: void checkAlterIsPossible(const AlterCommands & commands, ContextPtr context) const override; - void alter(const AlterCommands & params, ContextPtr context, TableLockHolder & table_lock_holder) override; + void alter(const AlterCommands & params, ContextPtr context, AlterLockHolder & table_lock_holder) override; std::optional totalRows(const Settings &) const override { diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index 62e0ccc76aa..304f84c02eb 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -91,7 +91,7 @@ public: IStorage::renameInMemory(new_table_id); } - void alter(const AlterCommands & params, ContextPtr context, TableLockHolder & alter_lock_holder) override + void alter(const AlterCommands & params, ContextPtr context, AlterLockHolder & alter_lock_holder) override { getNested()->alter(params, context, alter_lock_holder); IStorage::setInMemoryMetadata(getNested()->getInMemoryMetadata()); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ef8052de0ec..4e2cad668a1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4355,7 +4355,8 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer zookeeper->multi(requests); { - auto lock = lockForAlter(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + auto alter_lock_holder = lockForAlter(getSettings()->lock_acquire_timeout_for_background_operations); LOG_INFO(log, "Metadata changed in ZooKeeper. Applying changes locally."); auto metadata_diff = ReplicatedMergeTreeTableMetadata(*this, getInMemoryMetadataPtr()).checkAndFindDiff(metadata_from_entry); @@ -4427,7 +4428,7 @@ PartitionBlockNumbersHolder StorageReplicatedMergeTree::allocateBlockNumbersInAf void StorageReplicatedMergeTree::alter( - const AlterCommands & commands, ContextPtr query_context, TableLockHolder & table_lock_holder) + const AlterCommands & commands, ContextPtr query_context, AlterLockHolder & table_lock_holder) { assertNotReadonly(); @@ -4632,7 +4633,7 @@ void StorageReplicatedMergeTree::alter( } } - table_lock_holder.reset(); + table_lock_holder.unlock(); LOG_DEBUG(log, "Updated shared metadata nodes in ZooKeeper. Waiting for replicas to apply changes."); waitForLogEntryToBeProcessedIfNecessary(*alter_entry, query_context, "Some replicas doesn't finish metadata alter: "); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index bc03dbb45ae..d3d8463bffd 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -129,7 +129,7 @@ public: const Names & deduplicate_by_columns, ContextPtr query_context) override; - void alter(const AlterCommands & commands, ContextPtr query_context, TableLockHolder & table_lock_holder) override; + void alter(const AlterCommands & commands, ContextPtr query_context, AlterLockHolder & table_lock_holder) override; void mutate(const MutationCommands & commands, ContextPtr context) override; void waitMutation(const String & znode_name, size_t mutations_sync) const; diff --git a/src/Storages/TableLockHolder.h b/src/Storages/TableLockHolder.h index 6ec06a5e66c..36fda2c69bb 100644 --- a/src/Storages/TableLockHolder.h +++ b/src/Storages/TableLockHolder.h @@ -16,8 +16,6 @@ struct TableExclusiveLockHolder private: friend class IStorage; - /// Order is important. - TableLockHolder alter_lock; TableLockHolder drop_lock; }; From 2d559e098c4246726f5850d204e1bf5a257aefb2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 Oct 2021 14:06:09 +0300 Subject: [PATCH 103/173] Fix test --- .../queries/0_stateless/00652_replicated_mutations_zookeeper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh index d0e40e2ee62..f070e34f2a1 100755 --- a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh +++ b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh @@ -63,7 +63,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_cleaner_r2(x UInt32) ENGINE cleanup_delay_period_random_add = 0" # Insert some data -${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_cleaner_r1(x) VALUES (1), (2), (3), (4)" +${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_cleaner_r1(x) VALUES (1), (2), (3), (4), (5)" # Add some mutations and wait for their execution ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_cleaner_r1 DELETE WHERE x = 1 SETTINGS mutations_sync = 2" From 3b5c4b7fd269b35f8c6840fc59400dce6d796b0a Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 26 Oct 2021 16:33:58 +0300 Subject: [PATCH 104/173] Apply suggestions from code review Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/operations/settings/settings.md | 4 ++-- docs/ru/operations/settings/settings.md | 10 +++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 47e0a3006da..34053ae247a 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3849,8 +3849,8 @@ Enables or disables waiting for processing of asynchronous insertion. If enabled Possible values: -- 0 — Disabled. -- 1 — Enabled. +- 0 — Client returns `OK` even if the data is not yet inserted. +- 1 — Client returns `OK` only after the data is inserted. Default value: `1`. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index fb84913c0db..826bd6db77d 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3645,9 +3645,9 @@ SELECT * FROM positional_arguments ORDER BY 2,3; Включает или отключает асинхронные вставки для нескольких параллельных соединений. Работает только для вставок по протоколу HTTP. Обратите внимание, что при таких вставках дедупликация не производится. -Если включено, данные собираются в пачки перед вставкой в таблицу. Это позволяет производить мелкие и частые вставки в Clickhouse (до 15000 запросов в секунду) без промежуточных таблиц. +Если включено, данные собираются в пачки перед вставкой в таблицу. Это позволяет производить мелкие и частые вставки в ClickHouse (до 15000 запросов в секунду) без промежуточных таблиц. -Вставка данных происходит либо как только привышен [async_insert_max_data_size](#async-insert-max-data-size), либо через [async_insert_busy_timeout](#async-insert-busy-timeout) миллисекунд после первого запроса `INSERT`. Если [async_insert_stale_timeout](#async-insert-stale-timeout) задано ненулевое значение, то данные вставляются через `async_insert_stale_timeout` после последнего запроса. +Вставка данных происходит либо как только объем вставляемых данных превышает [async_insert_max_data_size](#async-insert-max-data-size), либо через [async_insert_busy_timeout](#async-insert-busy-timeout) миллисекунд после первого запроса `INSERT`. Если в [async_insert_stale_timeout](#async-insert-stale-timeout) задано ненулевое значение, то данные вставляются через `async_insert_stale_timeout` миллисекунд после последнего запроса. Если включен параметр [wait_for_async_insert](#wait-for-async-insert), каждый клиент ждет, пока данные будут сброшены в таблицу. Иначе запрос будет обработан почти моментально, даже если данные еще не вставлены. @@ -3664,7 +3664,7 @@ SELECT * FROM positional_arguments ORDER BY 2,3; Возможные значения: -- положительное целое число. +- Положительное целое число. - 0 — асинхронные вставки отключены. Значение по умолчанию: `16`. @@ -3675,8 +3675,8 @@ SELECT * FROM positional_arguments ORDER BY 2,3; Возможные значения: -- 0 — отключено. -- 1 — включено. +- 0 — клиент возвращает `OK` даже если вставка данных еще не завершена. +- 1 — клиент возвращает `OK` только после завершения вставки данных. Значение по умолчанию: `1`. From 901e5170a162b5ee10616e77a58e8dc200d9a0f2 Mon Sep 17 00:00:00 2001 From: gyuton Date: Tue, 26 Oct 2021 16:42:19 +0300 Subject: [PATCH 105/173] small fixes --- docs/en/operations/settings/settings.md | 8 ++++---- docs/en/sql-reference/statements/insert-into.md | 4 ++-- docs/ru/operations/settings/settings.md | 8 ++++---- docs/ru/sql-reference/statements/insert-into.md | 4 ++-- 4 files changed, 12 insertions(+), 12 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 34053ae247a..2287db1b6c0 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3821,7 +3821,7 @@ Enables or disables asynchronous inserts for multiple concurrent connections. Th If enabled, the data is combined into batches before the insertion into tables, so it is possible to do small and frequent insertions into ClickHouse (up to 15000 queries per second) without buffer tables. -The data is inserted either after the [async_insert_max_data_size](#async-insert-max-data-size) is exceeded or after [async_insert_busy_timeout](#async-insert-busy-timeout) milliseconds since the first `INSERT` query. If the [async_insert_stale_timeout](#async-insert-stale-timeout) is set to a non-zero value, the data is inserted after `async_insert_stale_timeout` milliseconds since the last query. +The data is inserted either after the [async_insert_max_data_size](#async-insert-max-data-size) is exceeded or after [async_insert_busy_timeout_ms](#async-insert-busy-timeout-ms) milliseconds since the first `INSERT` query. If the [async_insert_stale_timeout_ms](#async-insert-stale-timeout-ms) is set to a non-zero value, the data is inserted after `async_insert_stale_timeout_ms` milliseconds since the last query. If [wait_for_async_insert](#wait-for-async-insert) is enabled, every client will wait for the data to be processed and flushed to the table. Otherwise, the query would be processed almost instantly, even if the data is not inserted. @@ -3876,7 +3876,7 @@ Possible values: Default value: `1000000`. -## async_insert_busy_timeout {#async-insert-busy-timeout} +## async_insert_busy_timeout_ms {#async-insert-busy-timeout-ms} The maximum timeout in milliseconds since the first `INSERT` query before inserting collected data. @@ -3887,9 +3887,9 @@ Possible values: Default value: `200`. -## async_insert_stale_timeout {#async-insert-stale-timeout} +## async_insert_stale_timeout_ms {#async-insert-stale-timeout-ms} -The maximum timeout in milliseconds since the last `INSERT` query before dumping collected data. If enabled, the settings prolongs the [async_insert_busy_timeout](#async-insert-busy-timeout) with every `INSERT` query as long as [async_insert_max_data_size](#async-insert-max-data-size) is not exceeded. +The maximum timeout in milliseconds since the last `INSERT` query before dumping collected data. If enabled, the settings prolongs the [async_insert_busy_timeout_ms](#async-insert-busy-timeout-ms) with every `INSERT` query as long as [async_insert_max_data_size](#async-insert-max-data-size) is not exceeded. Possible values: diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index ad5ab705937..0a37a5f41ae 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -128,5 +128,5 @@ It's also possible to asynchronously insert data in small but frequent inserts f - [wait_for_async_insert](../../operations/settings/settings.md#wait-for-async-insert) - [wait_for_async_insert_timeout](../../operations/settings/settings.md#wait-for-async-insert-timeout) - [async_insert_max_data_size](../../operations/settings/settings.md#async-insert-max-data-size) -- [async_insert_busy_timeout](../../operations/settings/settings.md#async-insert-busy-timeout) -- [async_insert_stale_timeout](../../operations/settings/settings.md#async-insert-stale-timeout) +- [async_insert_busy_timeout_ms](../../operations/settings/settings.md#async-insert-busy-timeout-ms) +- [async_insert_stale_timeout_ms](../../operations/settings/settings.md#async-insert-stale-timeout-ms) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 826bd6db77d..ffe52d365d3 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3647,7 +3647,7 @@ SELECT * FROM positional_arguments ORDER BY 2,3; Если включено, данные собираются в пачки перед вставкой в таблицу. Это позволяет производить мелкие и частые вставки в ClickHouse (до 15000 запросов в секунду) без промежуточных таблиц. -Вставка данных происходит либо как только объем вставляемых данных превышает [async_insert_max_data_size](#async-insert-max-data-size), либо через [async_insert_busy_timeout](#async-insert-busy-timeout) миллисекунд после первого запроса `INSERT`. Если в [async_insert_stale_timeout](#async-insert-stale-timeout) задано ненулевое значение, то данные вставляются через `async_insert_stale_timeout` миллисекунд после последнего запроса. +Вставка данных происходит либо как только объем вставляемых данных превышает [async_insert_max_data_size](#async-insert-max-data-size), либо через [async_insert_busy_timeout_ms](#async-insert-busy-timeout-ms) миллисекунд после первого запроса `INSERT`. Если в [async_insert_stale_timeout_ms](#async-insert-stale-timeout-ms) задано ненулевое значение, то данные вставляются через `async_insert_stale_timeout_ms` миллисекунд после последнего запроса. Если включен параметр [wait_for_async_insert](#wait-for-async-insert), каждый клиент ждет, пока данные будут сброшены в таблицу. Иначе запрос будет обработан почти моментально, даже если данные еще не вставлены. @@ -3702,7 +3702,7 @@ SELECT * FROM positional_arguments ORDER BY 2,3; Значение по умолчанию: `1000000`. -## async_insert_busy_timeout {#async-insert-busy-timeout} +## async_insert_busy_timeout_ms {#async-insert-busy-timeout-ms} Максимальное время ожидания в миллисекундах после первого запроса `INSERT` и перед вставкой данных. @@ -3713,9 +3713,9 @@ SELECT * FROM positional_arguments ORDER BY 2,3; Значение по умолчанию: `200`. -## async_insert_stale_timeout {#async-insert-stale-timeout} +## async_insert_stale_timeout_ms {#async-insert-stale-timeout-ms} -Максимальное время ожидания в миллисекундах после последнего запроса `INSERT` и перед вставкой данных. Если установлено ненулевое значение, [async_insert_busy_timeout](#async-insert-busy-timeout) будет продлеваться с каждым запросом `INSERT`, пока не будет превышен [async_insert_max_data_size](#async-insert-max-data-size). +Максимальное время ожидания в миллисекундах после последнего запроса `INSERT` и перед вставкой данных. Если установлено ненулевое значение, [async_insert_busy_timeout_ms](#async-insert-busy-timeout-ms) будет продлеваться с каждым запросом `INSERT`, пока не будет превышен [async_insert_max_data_size](#async-insert-max-data-size). Возможные значения: diff --git a/docs/ru/sql-reference/statements/insert-into.md b/docs/ru/sql-reference/statements/insert-into.md index d7732cc58e1..7fc801b3b88 100644 --- a/docs/ru/sql-reference/statements/insert-into.md +++ b/docs/ru/sql-reference/statements/insert-into.md @@ -130,5 +130,5 @@ INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... - [wait_for_async_insert](../../operations/settings/settings.md#wait-for-async-insert) - [wait_for_async_insert_timeout](../../operations/settings/settings.md#wait-for-async-insert-timeout) - [async_insert_max_data_size](../../operations/settings/settings.md#async-insert-max-data-size) -- [async_insert_busy_timeout](../../operations/settings/settings.md#async-insert-busy-timeout) -- [async_insert_stale_timeout](../../operations/settings/settings.md#async-insert-stale-timeout) +- [async_insert_busy_timeout_ms](../../operations/settings/settings.md#async-insert-busy-timeout-ms) +- [async_insert_stale_timeout_ms](../../operations/settings/settings.md#async-insert-stale-timeout-ms) From 6be1e9834249a32ffed02f72ccc020a2ab14e831 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 26 Oct 2021 17:06:48 +0300 Subject: [PATCH 106/173] fix --- src/Storages/MergeTree/MergeTreeData.cpp | 5 ++++- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 15 +++++++++++---- src/Storages/StorageReplicatedMergeTree.cpp | 6 +++++- 3 files changed, 20 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 57729ef2eda..8dbe79e3cf7 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2692,7 +2692,10 @@ MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet(c void MergeTreeData::forgetPartAndMoveToDetached(const MergeTreeData::DataPartPtr & part_to_detach, const String & prefix, bool restore_covered) { - LOG_INFO(log, "Renaming {} to {}{} and forgetting it.", part_to_detach->relative_path, prefix, part_to_detach->name); + if (prefix.empty()) + LOG_INFO(log, "Renaming {} to {} and forgetting it.", part_to_detach->relative_path, part_to_detach->name); + else + LOG_INFO(log, "Renaming {} to {}_{} and forgetting it.", part_to_detach->relative_path, prefix, part_to_detach->name); auto lock = lockParts(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 985d54e8aee..2121a9dc09e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -187,13 +187,20 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) void ReplicatedMergeTreeQueue::createLogEntriesToFetchBrokenParts() { - std::lock_guard lock(state_mutex); - if (broken_parts_to_enqueue_fetches_on_loading.empty()) - return; + Strings broken_parts; + { + std::lock_guard lock(state_mutex); + broken_parts = broken_parts_to_enqueue_fetches_on_loading; + } - for (const auto & broken_part_name : broken_parts_to_enqueue_fetches_on_loading) + /// It will lock state_mutex + for (const auto & broken_part_name : broken_parts) storage.removePartAndEnqueueFetch(broken_part_name); + std::lock_guard lock(state_mutex); + /// broken_parts_to_enqueue_fetches_on_loading can be assigned only once on table startup, + /// so actually no race conditions are possible + assert(broken_parts == broken_parts_to_enqueue_fetches_on_loading); broken_parts_to_enqueue_fetches_on_loading.clear(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f7bccb7e823..053020f7007 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3352,7 +3352,11 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n fs::path(replica_path) / "queue/queue-", log_entry->toString(), zkutil::CreateMode::PersistentSequential)); - zookeeper->multi(ops); + auto results = zookeeper->multi(ops); + + String path_created = dynamic_cast(*results.back()).path_created; + log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1); + queue.insert(zookeeper, log_entry); } From d4d97d7b9a1f37ee59729080edfbfde55fb0a57a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Tue, 28 Sep 2021 16:24:06 +0800 Subject: [PATCH 107/173] add new function mapContainsKeyLike fixed error because of inconsistent offsets --- src/Functions/MatchImpl.h | 5 + src/Functions/like.cpp | 9 -- src/Functions/map.cpp | 122 ++++++++++++++++++ ...2030_function_mapContainsKeyLike.reference | 6 + .../02030_function_mapContainsKeyLike.sql | 12 ++ 5 files changed, 145 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02030_function_mapContainsKeyLike.reference create mode 100644 tests/queries/0_stateless/02030_function_mapContainsKeyLike.sql diff --git a/src/Functions/MatchImpl.h b/src/Functions/MatchImpl.h index 7dc0712023f..4849fe6e12d 100644 --- a/src/Functions/MatchImpl.h +++ b/src/Functions/MatchImpl.h @@ -419,4 +419,9 @@ struct MatchImpl } }; +struct NameLike +{ + static constexpr auto name = "like"; +}; + } diff --git a/src/Functions/like.cpp b/src/Functions/like.cpp index 1ac9a8d7dab..5d2d5de1be2 100644 --- a/src/Functions/like.cpp +++ b/src/Functions/like.cpp @@ -5,19 +5,10 @@ namespace DB { -namespace -{ - -struct NameLike -{ - static constexpr auto name = "like"; -}; using LikeImpl = MatchImpl; using FunctionLike = FunctionsStringSearch; -} - void registerFunctionLike(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 5517dced3e0..f84312ba6dc 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -15,6 +16,8 @@ #include #include #include "array/arrayIndex.h" +#include "Functions/MatchImpl.h" +#include "Functions/FunctionsStringSearch.h" namespace DB @@ -28,6 +31,9 @@ namespace ErrorCodes namespace { +using LikeImpl = MatchImpl; +using FunctionLike = FunctionsStringSearch; + // map(x, y, ...) is a function that allows you to make key-value pair class FunctionMap : public IFunction { @@ -274,6 +280,121 @@ public: } }; +class FunctionMapContainsKeyLike : public IFunction +{ +public: + static constexpr auto name = "mapContainsKeyLike"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + String getName() const override { return name; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*info*/) const override { return true; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + bool is_const = isColumnConst(*arguments[0].column); + const ColumnMap * col_map = is_const ? checkAndGetColumnConstData(arguments[0].column.get()) + : checkAndGetColumn(arguments[0].column.get()); + const DataTypeMap * map_type = checkAndGetDataType(arguments[0].type.get()); + if (!col_map || !map_type) + throw Exception{"First argument for function " + getName() + " must be a map", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_res = col_res->getData(); + + if (input_rows_count == 0) + return col_res; + + vec_res.resize(input_rows_count); + + const auto & column_array = typeid_cast(col_map->getNestedColumn()); + const auto & column_tuple = typeid_cast(column_array.getData()); + + const ColumnString * column_string = checkAndGetColumn(column_tuple.getColumn(0)); + const ColumnFixedString * column_fixed_string = checkAndGetColumn(column_tuple.getColumn(0)); + + FunctionLike func_like; + + for (size_t row = 0; row < input_rows_count; row++) + { + size_t element_start_row = row != 0 ? column_array.getOffsets()[row-1] : 0; + size_t elem_size = column_array.getOffsets()[row]- element_start_row; + + ColumnPtr sub_map_column; + DataTypePtr data_type; + + //The keys of one row map will be processed as a single ColumnString + if (column_string) + { + sub_map_column = column_string->cut(element_start_row, elem_size); + data_type = std::make_shared(); + } + else + { + sub_map_column = column_fixed_string->cut(element_start_row, elem_size); + data_type = std::make_shared(checkAndGetColumn(sub_map_column.get())->getN()); + } + + size_t col_key_size = sub_map_column->size(); + auto column = is_const? ColumnConst::create(std::move(sub_map_column), std::move(col_key_size)) : std::move(sub_map_column); + + ColumnsWithTypeAndName new_arguments = + { + { + column, + data_type, + "" + }, + arguments[1] + }; + + auto res = func_like.executeImpl(new_arguments, result_type, input_rows_count); + + const auto & container = checkAndGetColumn(res.get())->getData(); + + bool exist = 0; + for (auto iter = container.begin(); iter != container.end(); iter++) + { + if (*iter == 1) + { + exist = 1; + break; + } + } + + vec_res[row] = exist; + } + + return col_res; + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() != 2) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 2", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const DataTypeMap * map_type = checkAndGetDataType(arguments[0].type.get()); + const DataTypeString * pattern_type = checkAndGetDataType(arguments[1].type.get()); + + if (!map_type) + throw Exception{"First argument for function " + getName() + " must be a Map", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + if (!pattern_type) + throw Exception{"Second argument for function " + getName() + " must be String", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + if (!isStringOrFixedString(map_type->getKeyType())) + throw Exception{"Key type of map for function " + getName() + " must be `String` or `FixedString`", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return std::make_shared(); + } + + size_t getNumberOfArguments() const override { return 2; } + + bool useDefaultImplementationForConstants() const override { return true; } +}; + } void registerFunctionsMap(FunctionFactory & factory) @@ -282,6 +403,7 @@ void registerFunctionsMap(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); } } diff --git a/tests/queries/0_stateless/02030_function_mapContainsKeyLike.reference b/tests/queries/0_stateless/02030_function_mapContainsKeyLike.reference new file mode 100644 index 00000000000..eb8bc83384e --- /dev/null +++ b/tests/queries/0_stateless/02030_function_mapContainsKeyLike.reference @@ -0,0 +1,6 @@ +1 {'1-K1':'1-V1','1-K2':'1-V2'} +1 {'1-K1':'1-V1','1-K2':'1-V2'} +2 {'2-K1':'2-V1','2-K2':'2-V2'} +4 {'4-K1':'4-V1','4-K2':'4-V2'} +5 {'5-K1':'5-V1','5-K2':'5-V2'} +6 {'6-K1':'6-V1','6-K2':'6-V2'} diff --git a/tests/queries/0_stateless/02030_function_mapContainsKeyLike.sql b/tests/queries/0_stateless/02030_function_mapContainsKeyLike.sql new file mode 100644 index 00000000000..7d9722b4c90 --- /dev/null +++ b/tests/queries/0_stateless/02030_function_mapContainsKeyLike.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS map_containsKeyLike_test; + +CREATE TABLE map_containsKeyLike_test (id UInt32, map Map(String, String)) Engine=MergeTree() ORDER BY id settings index_granularity=2; + +INSERT INTO map_containsKeyLike_test VALUES (1, {'1-K1':'1-V1','1-K2':'1-V2'}),(2,{'2-K1':'2-V1','2-K2':'2-V2'}); +INSERT INTO map_containsKeyLike_test VALUES (3, {'3-K1':'3-V1','3-K2':'3-V2'}),(4, {'4-K1':'4-V1','4-K2':'4-V2'}); +INSERT INTO map_containsKeyLike_test VALUES (5, {'5-K1':'5-V1','5-K2':'5-V2'}),(6, {'6-K1':'6-V1','6-K2':'6-V2'}); + +SELECT id, map FROM map_containsKeyLike_test WHERE mapContainsKeyLike(map, '1-%') = 1; +SELECT id, map FROM map_containsKeyLike_test WHERE mapContainsKeyLike(map, '3-%') = 0 order by id; + +DROP TABLE map_containsKeyLike_test; From 1caff8a81c713d244f9d51bd7984c822ca036af3 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Tue, 26 Oct 2021 22:12:21 +0530 Subject: [PATCH 108/173] fixed compile error --- src/Columns/ColumnsCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index 2be9b18de4e..3b27ca9bd71 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -41,7 +41,7 @@ UInt64 Bytes64MaskToBits64Mask(const UInt8 * bytes64) const UInt8 * pos = bytes64; const UInt8 * end = pos + 64; for (; pos < end; ++pos) - res |= (*pos == 0)<<(pos-bytes64)); + res |= ((*pos == 0)<<(pos-bytes64)); #endif return ~res; } From bc0a9c69848dd24b66b42a059c708827bb6dd58a Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Tue, 26 Oct 2021 20:09:13 +0300 Subject: [PATCH 109/173] Update index.md --- docs/en/sql-reference/operators/index.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index d23c81e2aca..61a8341db14 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -73,11 +73,11 @@ ClickHouse transforms operators to their corresponding functions at the query pa `a = ANY (subquery)` – The `in(a, subquery)` function. -`a != ANY (subquery)` – The `notIn(a, subquery)` function. +`a != ANY (subquery)` – The same as `a NOT IN (SELECT singleValueOrNull(*) FROM subquery)`. -`a = ALL (subquery)` – The same as `a IN (SELECT singleValueOrNull(*) FROM subquery)`. +`a = ALL (subquery)` – The same as `a IN (SELECT singleValueOrNull(*) FROM subquery)`. -`a != ALL (subquery)` – The same as `a NOT IN (subquery)`. +`a != ALL (subquery)` – The `notIn(a, subquery)` function. **Examples** From da7d491125495f8d91a3bc159666b34b488f2169 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Tue, 26 Oct 2021 20:10:30 +0300 Subject: [PATCH 110/173] Update index.md --- docs/ru/sql-reference/operators/index.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/ru/sql-reference/operators/index.md b/docs/ru/sql-reference/operators/index.md index 19119f641ef..2d67eb1348b 100644 --- a/docs/ru/sql-reference/operators/index.md +++ b/docs/ru/sql-reference/operators/index.md @@ -72,13 +72,13 @@ toc_title: "Операторы" `a GLOBAL NOT IN ...` - функция `globalNotIn(a, b)` -`a = ANY (subquery)` – функция `in(a, subquery)` +`a = ANY (subquery)` – функция `in(a, subquery)`. -`a != ANY (subquery)` – функция `notIn(a, subquery)` +`a != ANY (subquery)` – равнозначно `a NOT IN (SELECT singleValueOrNull(*) FROM subquery)`. -`a = ALL (subquery)` – равнозначно `a IN (SELECT singleValueOrNull(*) FROM subquery)` +`a = ALL (subquery)` – равнозначно `a IN (SELECT singleValueOrNull(*) FROM subquery)`. -`a != ALL (subquery)` – равнозначно `a NOT IN (subquery)` +`a != ALL (subquery)` – функция `notIn(a, subquery)`. **Примеры** From 403ff98baa41adff67127871fa65c3a99700e15d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 26 Oct 2021 20:15:29 +0300 Subject: [PATCH 111/173] Revert "Improve usability of `remote_url_allow_hosts`" --- src/Common/RemoteHostFilter.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/RemoteHostFilter.cpp b/src/Common/RemoteHostFilter.cpp index ba7f163fd16..73c84364f3c 100644 --- a/src/Common/RemoteHostFilter.cpp +++ b/src/Common/RemoteHostFilter.cpp @@ -18,14 +18,14 @@ void RemoteHostFilter::checkURL(const Poco::URI & uri) const { if (!checkForDirectEntry(uri.getHost()) && !checkForDirectEntry(uri.getHost() + ":" + toString(uri.getPort()))) - throw Exception("URL \"" + uri.toString() + "\" is not allowed in configuration file, see ", ErrorCodes::UNACCEPTABLE_URL); + throw Exception("URL \"" + uri.toString() + "\" is not allowed in config.xml", ErrorCodes::UNACCEPTABLE_URL); } void RemoteHostFilter::checkHostAndPort(const std::string & host, const std::string & port) const { if (!checkForDirectEntry(host) && !checkForDirectEntry(host + ":" + port)) - throw Exception("URL \"" + host + ":" + port + "\" is not allowed in configuration file, see ", ErrorCodes::UNACCEPTABLE_URL); + throw Exception("URL \"" + host + ":" + port + "\" is not allowed in config.xml", ErrorCodes::UNACCEPTABLE_URL); } void RemoteHostFilter::setValuesFromConfig(const Poco::Util::AbstractConfiguration & config) From aae30a9e37d71fa84d828cf70352cad7f7f4507c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 26 Oct 2021 20:30:34 +0300 Subject: [PATCH 112/173] Fix fuzzer build (#30344) --- CMakeLists.txt | 2 -- contrib/protobuf-cmake/CMakeLists.txt | 10 ++++++++++ docker/packager/other/fuzzer.sh | 12 ++++++------ src/Interpreters/fuzzers/CMakeLists.txt | 1 - src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt | 2 +- 5 files changed, 17 insertions(+), 10 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 1790ddc0451..ede70b09d94 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -149,8 +149,6 @@ if (ENABLE_FUZZING) set (ENABLE_JEMALLOC 0) set (ENABLE_CHECK_HEAVY_BUILDS 1) set (GLIBC_COMPATIBILITY OFF) - set (ENABLE_PROTOBUF ON) - set (USE_INTERNAL_PROTOBUF_LIBRARY ON) endif() # Global libraries diff --git a/contrib/protobuf-cmake/CMakeLists.txt b/contrib/protobuf-cmake/CMakeLists.txt index 0ceb72cfbd6..37215b91dbb 100644 --- a/contrib/protobuf-cmake/CMakeLists.txt +++ b/contrib/protobuf-cmake/CMakeLists.txt @@ -17,6 +17,16 @@ endif () add_subdirectory("${protobuf_SOURCE_DIR}/cmake" "${protobuf_BINARY_DIR}") +if (ENABLE_FUZZING) + # `protoc` will be built with sanitizer and it could fail during ClickHouse build + # It easily reproduces in oss-fuzz building pipeline + # To avoid this we can try to build `protoc` without any sanitizer with option `-fno-sanitize=all`, but + # it this case we will face with linker errors, because libcxx still will be built with sanitizer + # So, we can simply suppress all of these failures with a combination this flag and an environment variable + # export MSAN_OPTIONS=exit_code=0 + target_compile_options(protoc PRIVATE "-fsanitize-recover=all") +endif() + # We don't want to stop compilation on warnings in protobuf's headers. # The following line overrides the value assigned by the command target_include_directories() in libprotobuf.cmake set_property(TARGET libprotobuf PROPERTY INTERFACE_SYSTEM_INCLUDE_DIRECTORIES "${protobuf_SOURCE_DIR}/src") diff --git a/docker/packager/other/fuzzer.sh b/docker/packager/other/fuzzer.sh index 5eec51f9625..baaf3c4bedf 100755 --- a/docker/packager/other/fuzzer.sh +++ b/docker/packager/other/fuzzer.sh @@ -12,19 +12,19 @@ printenv rm -f CMakeCache.txt read -ra CMAKE_FLAGS <<< "${CMAKE_FLAGS:-}" # Hope, that the most part of files will be in cache, so we just link new executables -cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_C_COMPILER="$CC" -DCMAKE_CXX_COMPILER="$CXX" -DENABLE_CLICKHOUSE_ODBC_BRIDGE=OFF \ - -DENABLE_LIBRARIES=0 -DENABLE_SSL=1 -DUSE_INTERNAL_SSL_LIBRARY=1 -DUSE_UNWIND=ON -DENABLE_EMBEDDED_COMPILER=0 \ - -DENABLE_EXAMPLES=0 -DENABLE_UTILS=0 -DENABLE_THINLTO=0 "-DSANITIZE=$SANITIZER" \ - -DENABLE_FUZZING=1 -DFUZZER='libfuzzer' -DENABLE_TCMALLOC=0 -DENABLE_JEMALLOC=0 \ - -DENABLE_CHECK_HEAVY_BUILDS=1 -DGLIBC_COMPATIBILITY=OFF "${CMAKE_FLAGS[@]}" .. +# Please, add or change flags directly in cmake +cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_C_COMPILER="$CC" -DCMAKE_CXX_COMPILER="$CXX" \ + -DSANITIZE="$SANITIZER" -DENABLE_FUZZING=1 -DFUZZER='libfuzzer' -DENABLE_PROTOBUF=1 -DUSE_INTERNAL_PROTOBUF_LIBRARY=1 "${CMAKE_FLAGS[@]}" .. FUZZER_TARGETS=$(find ../src -name '*_fuzzer.cpp' -execdir basename {} .cpp ';' | tr '\n' ' ') +NUM_JOBS=$(($(nproc || grep -c ^processor /proc/cpuinfo))) + mkdir -p /output/fuzzers for FUZZER_TARGET in $FUZZER_TARGETS do # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. - ninja $NINJA_FLAGS $FUZZER_TARGET + ninja $NINJA_FLAGS $FUZZER_TARGET -j $NUM_JOBS # Find this binary in build directory and strip it FUZZER_PATH=$(find ./src -name "$FUZZER_TARGET") strip --strip-unneeded "$FUZZER_PATH" diff --git a/src/Interpreters/fuzzers/CMakeLists.txt b/src/Interpreters/fuzzers/CMakeLists.txt index dfe892b5249..f8d4cf224db 100644 --- a/src/Interpreters/fuzzers/CMakeLists.txt +++ b/src/Interpreters/fuzzers/CMakeLists.txt @@ -7,4 +7,3 @@ target_link_libraries(execute_query_fuzzer PRIVATE clickhouse_dictionaries clickhouse_dictionaries_embedded ${LIB_FUZZING_ENGINE}) - diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt index 6f6b0d07661..d5e6c944852 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -45,4 +45,4 @@ set_source_files_properties("${PROTO_SRCS}" "out.cpp" PROPERTIES COMPILE_FLAGS " target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${Protobuf_INCLUDE_DIR}" "${CMAKE_CURRENT_BINARY_DIR}") target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${LibProtobufMutator_SOURCE_DIR}") target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${LibProtobufMutator_SOURCE_DIR}/src") -target_link_libraries(codegen_select_fuzzer PRIVATE protobuf-mutator dbms ${LIB_FUZZING_ENGINE}) +target_link_libraries(codegen_select_fuzzer PRIVATE protobuf-mutator ${Protobuf_LIBRARY} ${Protobuf_PROTOC_LIBRARY} dbms ${LIB_FUZZING_ENGINE}) From fb31410926fd3d3097b983e928468274f4f1d2d6 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Wed, 27 Oct 2021 00:05:47 +0530 Subject: [PATCH 113/173] change to inline for bitmask generator --- src/CMakeLists.txt | 3 ++- src/Columns/ColumnsCommon.cpp | 38 +------------------------------- src/Columns/ColumnsCommon.h | 41 ++++++++++++++++++++++++++++++++--- 3 files changed, 41 insertions(+), 41 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 9e1f25c21d9..5fc15b1609d 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -290,12 +290,13 @@ target_link_libraries (clickhouse_common_io ) # Use X86 AVX2/AVX512 instructions to accelerate filter operations +# Add flag:-force-vector-width=16 due to clang13 not support auto vectorize with AVX512, maybe remove it in future. set_source_files_properties( Columns/ColumnFixedString.cpp Columns/ColumnsCommon.cpp Columns/ColumnVector.cpp Columns/ColumnDecimal.cpp - PROPERTIES COMPILE_FLAGS "${X86_INTRINSICS_FLAGS}") + PROPERTIES COMPILE_FLAGS "${X86_INTRINSICS_FLAGS} -mprefer-vector-width=256") if(RE2_LIBRARY) target_link_libraries(clickhouse_common_io PUBLIC ${RE2_LIBRARY}) diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index 3b27ca9bd71..315376cadec 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -1,6 +1,4 @@ -#ifdef __SSE2__ - #include -#endif + #include #include @@ -12,40 +10,6 @@ namespace DB { -/// Transform 64-byte mask to 64-bit mask -UInt64 Bytes64MaskToBits64Mask(const UInt8 * bytes64) -{ -#if defined(__AVX512F__) && defined(__AVX512BW__) - static const __m512i zero64 = _mm512_setzero_epi32(); - UInt64 res = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(bytes64)), zero64, _MM_CMPINT_EQ); -#elif defined(__AVX__) && defined(__AVX2__) - static const __m256i zero32 = _mm256_setzero_si256(); - UInt64 res = - (static_cast(_mm256_movemask_epi8(_mm256_cmpeq_epi8( - _mm256_loadu_si256(reinterpret_cast(bytes64)), zero32))) & 0xffffffff) - | (static_cast(_mm256_movemask_epi8(_mm256_cmpeq_epi8( - _mm256_loadu_si256(reinterpret_cast(bytes64+32)), zero32))) << 32); -#elif defined(__SSE2__) && defined(__POPCNT__) - static const __m128i zero16 = _mm_setzero_si128(); - UInt64 res = - (static_cast(_mm_movemask_epi8(_mm_cmpeq_epi8( - _mm_loadu_si128(reinterpret_cast(bytes64)), zero16))) & 0xffff) - | ((static_cast(_mm_movemask_epi8(_mm_cmpeq_epi8( - _mm_loadu_si128(reinterpret_cast(bytes64 + 16)), zero16))) << 16) & 0xffff0000) - | ((static_cast(_mm_movemask_epi8(_mm_cmpeq_epi8( - _mm_loadu_si128(reinterpret_cast(bytes64 + 32)), zero16))) << 32) & 0xffff00000000) - | ((static_cast(_mm_movemask_epi8(_mm_cmpeq_epi8( - _mm_loadu_si128(reinterpret_cast(bytes64 + 48)), zero16))) << 48) & 0xffff000000000000); -#else - UInt64 res = 0; - const UInt8 * pos = bytes64; - const UInt8 * end = pos + 64; - for (; pos < end; ++pos) - res |= ((*pos == 0)<<(pos-bytes64)); -#endif - return ~res; -} - size_t countBytesInFilter(const UInt8 * filt, size_t sz) { size_t count = 0; diff --git a/src/Columns/ColumnsCommon.h b/src/Columns/ColumnsCommon.h index baf26deb00c..c2feb0b31c9 100644 --- a/src/Columns/ColumnsCommon.h +++ b/src/Columns/ColumnsCommon.h @@ -2,8 +2,12 @@ #include #include - - +#ifdef __SSE2__ +#include +#endif +#if defined(__AVX512F__) || defined(__AVX512BW__) || defined(__AVX__) || defined(__AVX2__) +#include +#endif /// Common helper methods for implementation of different columns. namespace DB @@ -16,7 +20,38 @@ namespace ErrorCodes } /// Transform 64-byte mask to 64-bit mask -UInt64 Bytes64MaskToBits64Mask(const UInt8 * bytes64); +inline UInt64 Bytes64MaskToBits64Mask(const UInt8 * bytes64) +{ +#if defined(__AVX512F__) && defined(__AVX512BW__) + static const __m512i zero64 = _mm512_setzero_epi32(); + UInt64 res = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(bytes64)), zero64, _MM_CMPINT_GT); +#elif defined(__AVX__) && defined(__AVX2__) + static const __m256i zero32 = _mm256_setzero_si256(); + UInt64 res = + (static_cast(_mm256_movemask_epi8(_mm256_cmpgt_epi8( + _mm256_loadu_si256(reinterpret_cast(bytes64)), zero32))) & 0xffffffff) + | (static_cast(_mm256_movemask_epi8(_mm256_cmpgt_epi8( + _mm256_loadu_si256(reinterpret_cast(bytes64+32)), zero32))) << 32); +#elif defined(__SSE2__) && defined(__POPCNT__) + static const __m128i zero16 = _mm_setzero_si128(); + UInt64 res = + static_cast(_mm_movemask_epi8(_mm_cmpgt_epi8( + _mm_loadu_si128(reinterpret_cast(bytes64)), zero16))) + | (static_cast(_mm_movemask_epi8(_mm_cmpgt_epi8( + _mm_loadu_si128(reinterpret_cast(bytes64 + 16)), zero16))) << 16) + | (static_cast(_mm_movemask_epi8(_mm_cmpgt_epi8( + _mm_loadu_si128(reinterpret_cast(bytes64 + 32)), zero16))) << 32) + | (static_cast(_mm_movemask_epi8(_mm_cmpgt_epi8( + _mm_loadu_si128(reinterpret_cast(bytes64 + 48)), zero16))) << 48); +#else + UInt64 res = 0; + const UInt8 * pos = bytes64; + const UInt8 * end = pos + 64; + for (; pos < end; ++pos) + res |= ((*pos != 0)<<(pos-bytes64)); +#endif + return res; +} /// Counts how many bytes of `filt` are greater than zero. size_t countBytesInFilter(const UInt8 * filt, size_t sz); size_t countBytesInFilter(const IColumn::Filter & filt); From 874f7a505263aabf3f5bb949d707f065fa2526ef Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 22 Oct 2021 00:12:30 +0300 Subject: [PATCH 114/173] Add a test with optimized columns from PREWHERE --- .../02021_prewhere_column_optimization.reference | 7 +++++++ .../0_stateless/02021_prewhere_column_optimization.sql | 10 ++++++++++ 2 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/02021_prewhere_column_optimization.reference create mode 100644 tests/queries/0_stateless/02021_prewhere_column_optimization.sql diff --git a/tests/queries/0_stateless/02021_prewhere_column_optimization.reference b/tests/queries/0_stateless/02021_prewhere_column_optimization.reference new file mode 100644 index 00000000000..7f97e6efd69 --- /dev/null +++ b/tests/queries/0_stateless/02021_prewhere_column_optimization.reference @@ -0,0 +1,7 @@ +-- { echoOn } +select * from data_02021 prewhere 1 or ignore(key); +1 +select * from data_02021 prewhere 1 or ignore(key) where key = 1; +1 +select * from data_02021 prewhere 0 or ignore(key); +select * from data_02021 prewhere 0 or ignore(key) where key = 1; diff --git a/tests/queries/0_stateless/02021_prewhere_column_optimization.sql b/tests/queries/0_stateless/02021_prewhere_column_optimization.sql new file mode 100644 index 00000000000..4fe8b912c3f --- /dev/null +++ b/tests/queries/0_stateless/02021_prewhere_column_optimization.sql @@ -0,0 +1,10 @@ +drop table if exists data_02021; +create table data_02021 (key Int) engine=MergeTree() order by key; +insert into data_02021 values (1); +-- { echoOn } +select * from data_02021 prewhere 1 or ignore(key); +select * from data_02021 prewhere 1 or ignore(key) where key = 1; +select * from data_02021 prewhere 0 or ignore(key); +select * from data_02021 prewhere 0 or ignore(key) where key = 1; +-- { echoOff } +drop table data_02021; From 248b689067e3c90d68673ae86678389992ae4316 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 22 Oct 2021 00:15:59 +0300 Subject: [PATCH 115/173] ActionsDag: add allow_remove_inputs for other variants of removeUnusedActions() --- src/Interpreters/ActionsDAG.cpp | 8 ++++---- src/Interpreters/ActionsDAG.h | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index ea9ee2ce542..56266d50e9d 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -325,7 +325,7 @@ std::string ActionsDAG::dumpNames() const return out.str(); } -void ActionsDAG::removeUnusedActions(const NameSet & required_names) +void ActionsDAG::removeUnusedActions(const NameSet & required_names, bool allow_remove_inputs) { NodeRawConstPtrs required_nodes; required_nodes.reserve(required_names.size()); @@ -349,10 +349,10 @@ void ActionsDAG::removeUnusedActions(const NameSet & required_names) } index.swap(required_nodes); - removeUnusedActions(); + removeUnusedActions(allow_remove_inputs); } -void ActionsDAG::removeUnusedActions(const Names & required_names) +void ActionsDAG::removeUnusedActions(const Names & required_names, bool allow_remove_inputs) { NodeRawConstPtrs required_nodes; required_nodes.reserve(required_names.size()); @@ -372,7 +372,7 @@ void ActionsDAG::removeUnusedActions(const Names & required_names) } index.swap(required_nodes); - removeUnusedActions(); + removeUnusedActions(allow_remove_inputs); } void ActionsDAG::removeUnusedActions(bool allow_remove_inputs) diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 8111bcb123c..9baab1231d4 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -162,8 +162,8 @@ public: bool isInputProjected() const { return project_input; } bool isOutputProjected() const { return projected_output; } - void removeUnusedActions(const Names & required_names); - void removeUnusedActions(const NameSet & required_names); + void removeUnusedActions(const Names & required_names, bool allow_remove_inputs = true); + void removeUnusedActions(const NameSet & required_names, bool allow_remove_inputs = true); NameSet foldActionsByProjection( const NameSet & required_columns, From dd7bffcaf96a5d143d77d65e3d0510bf53c0bfd0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 26 Oct 2021 22:56:12 +0300 Subject: [PATCH 116/173] ActionsDAG: add allow_constant_folding flag for removeUnusedActions() --- src/Interpreters/ActionsDAG.cpp | 14 +++++++------- src/Interpreters/ActionsDAG.h | 6 +++--- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 56266d50e9d..d0b360dda82 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -325,7 +325,7 @@ std::string ActionsDAG::dumpNames() const return out.str(); } -void ActionsDAG::removeUnusedActions(const NameSet & required_names, bool allow_remove_inputs) +void ActionsDAG::removeUnusedActions(const NameSet & required_names, bool allow_remove_inputs, bool allow_constant_folding) { NodeRawConstPtrs required_nodes; required_nodes.reserve(required_names.size()); @@ -349,10 +349,10 @@ void ActionsDAG::removeUnusedActions(const NameSet & required_names, bool allow_ } index.swap(required_nodes); - removeUnusedActions(allow_remove_inputs); + removeUnusedActions(allow_remove_inputs, allow_constant_folding); } -void ActionsDAG::removeUnusedActions(const Names & required_names, bool allow_remove_inputs) +void ActionsDAG::removeUnusedActions(const Names & required_names, bool allow_remove_inputs, bool allow_constant_folding) { NodeRawConstPtrs required_nodes; required_nodes.reserve(required_names.size()); @@ -372,10 +372,10 @@ void ActionsDAG::removeUnusedActions(const Names & required_names, bool allow_re } index.swap(required_nodes); - removeUnusedActions(allow_remove_inputs); + removeUnusedActions(allow_remove_inputs, allow_constant_folding); } -void ActionsDAG::removeUnusedActions(bool allow_remove_inputs) +void ActionsDAG::removeUnusedActions(bool allow_remove_inputs, bool allow_constant_folding) { std::unordered_set visited_nodes; std::stack stack; @@ -406,9 +406,9 @@ void ActionsDAG::removeUnusedActions(bool allow_remove_inputs) auto * node = stack.top(); stack.pop(); - if (!node->children.empty() && node->column && isColumnConst(*node->column)) + /// Constant folding. + if (allow_constant_folding && !node->children.empty() && node->column && isColumnConst(*node->column)) { - /// Constant folding. node->type = ActionsDAG::ActionType::COLUMN; for (const auto & child : node->children) diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 9baab1231d4..95189416cc3 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -162,8 +162,8 @@ public: bool isInputProjected() const { return project_input; } bool isOutputProjected() const { return projected_output; } - void removeUnusedActions(const Names & required_names, bool allow_remove_inputs = true); - void removeUnusedActions(const NameSet & required_names, bool allow_remove_inputs = true); + void removeUnusedActions(const Names & required_names, bool allow_remove_inputs = true, bool allow_constant_folding = true); + void removeUnusedActions(const NameSet & required_names, bool allow_remove_inputs = true, bool allow_constant_folding = true); NameSet foldActionsByProjection( const NameSet & required_columns, @@ -271,7 +271,7 @@ public: private: Node & addNode(Node node); - void removeUnusedActions(bool allow_remove_inputs = true); + void removeUnusedActions(bool allow_remove_inputs = true, bool allow_constant_folding = true); #if USE_EMBEDDED_COMPILER void compileFunctions(size_t min_count_to_compile_expression, const std::unordered_set & lazy_executed_nodes = {}); From fbf98bea0ba56440b973fabbfe755aeb13f078ef Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 2 Sep 2021 08:35:30 +0300 Subject: [PATCH 117/173] Fix "Column is not under aggregate function and not in GROUP BY" with PREWHERE In 21.7 the function column type is unknown: (lldb) p tmp_actions_dag->dumpDAG() (std::string) $14 = "0 : INPUT () (no column) Int32 key\n1 : COLUMN () Const(UInt8) UInt8 1\n2 : FUNCTION (0) (no column) UInt8 ignore(key) [ignore]\n3 : FUNCTION (1, 2) (no column) UInt8 or(1, ignore(key)) [or]\nIndex: 0 1 2 3\n" ^^^^^^^^^^^ While in 21.8+ it is const: (lldb) p tmp_actions_dag->dumpDAG() (std::string) $22 = "0 : INPUT () (no column) Int32 key\n1 : COLUMN () Const(UInt8) UInt8 1\n2 : FUNCTION (0) (no column) UInt8 ignore(key) [ignore]\n3 : FUNCTION (1, 2) Const(UInt8) UInt8 or(1, ignore(key)) [or]\nIndex: 0 1 2 3\n" ^^^^^ This is after getRootActions() in SelectQueryExpressionAnalyzer::appendPrewhere() v1: fix for SELECT only, but breaks virtual columns v2: hacky fix, that also touches MergeTree code v3: allow_remove_inputs=false v4: allow_constant_folding=false instead --- src/Interpreters/ExpressionAnalyzer.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 501fc9b4b1d..d40a92972b2 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1045,7 +1045,12 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( /// Remove unused source_columns from prewhere actions. auto tmp_actions_dag = std::make_shared(sourceColumns()); getRootActions(select_query->prewhere(), only_types, tmp_actions_dag); - tmp_actions_dag->removeUnusedActions(NameSet{prewhere_column_name}); + /// Constants cannot be removed since they can be used in other parts of the query. + /// And if they are not used anywhere, except PREWHERE, they will be removed on the next step. + tmp_actions_dag->removeUnusedActions( + NameSet{prewhere_column_name}, + /* allow_remove_inputs= */ true, + /* allow_constant_folding= */ false); auto required_columns = tmp_actions_dag->getRequiredColumnsNames(); NameSet required_source_columns(required_columns.begin(), required_columns.end()); From b764b6ce216166aa0b571430c40fe0b5d1d17153 Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Tue, 26 Oct 2021 15:08:05 -0600 Subject: [PATCH 118/173] Update hardware page colors (#30719) Replaces yellow with a light gray --- website/benchmark/benchmark.js | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/website/benchmark/benchmark.js b/website/benchmark/benchmark.js index 8fb2693aa97..bd5ec3c083e 100644 --- a/website/benchmark/benchmark.js +++ b/website/benchmark/benchmark.js @@ -58,7 +58,7 @@ function generate_selectors(elem) { var available = available_systems_for_current_data_size.indexOf(systems[i]) != -1; var button_class = 'btn-outline-dark'; if (system_kinds[i] == 'cloud' || system_kinds[i] == 'vps') { - button_class = 'btn-outline-primary'; + button_class = 'btn-light'; } else if (system_kinds[i] == 'desktop' || system_kinds[i] == 'laptop') { button_class = 'btn-outline-secondary'; }; @@ -424,11 +424,11 @@ function generate_diagram() { } - table_row += '